diff --git a/.gitignore b/.gitignore index cd9f90d55932c..4f177c82ae5e0 100644 --- a/.gitignore +++ b/.gitignore @@ -7,7 +7,7 @@ sbt/*.jar .settings .cache -.mima-excludes +.generated-mima* /build/ work/ out/ @@ -18,6 +18,7 @@ conf/java-opts conf/spark-env.sh conf/streaming-env.sh conf/log4j.properties +conf/spark-defaults.conf docs/_site docs/api target/ @@ -48,3 +49,10 @@ unit-tests.log /lib/ rat-results.txt scalastyle.txt +conf/*.conf + +# For Hive +metastore_db/ +metastore/ +warehouse/ +TempStatsStore/ diff --git a/.rat-excludes b/.rat-excludes index 85bfad60fcadc..15344dfb292db 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -3,6 +3,7 @@ target .project .classpath .mima-excludes +.generated-mima-excludes .rat-excludes .*md derby.log @@ -11,6 +12,7 @@ RELEASE control docs fairscheduler.xml.template +spark-defaults.conf.template log4j.properties log4j.properties.template metrics.properties.template @@ -20,6 +22,7 @@ spark-env.sh.template log4j-defaults.properties sorttable.js .*txt +.*json .*data .*log cloudpickle.py @@ -40,3 +43,11 @@ work golden test.out/* .*iml +service.properties +db.lck +build/* +dist/* +.*out +.*ipr +.*iws +logs diff --git a/LICENSE b/LICENSE index 1c1c2c0255fa9..383f079df8c8b 100644 --- a/LICENSE +++ b/LICENSE @@ -428,3 +428,106 @@ LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON A THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +======================================================================== +For colt: +======================================================================== + +Copyright (c) 1999 CERN - European Organization for Nuclear Research. +Permission to use, copy, modify, distribute and sell this software and its documentation for any purpose is hereby granted without fee, provided that the above copyright notice appear in all copies and that both that copyright notice and this permission notice appear in supporting documentation. CERN makes no representations about the suitability of this software for any purpose. It is provided "as is" without expressed or implied warranty. + +Packages hep.aida.* + +Written by Pavel Binko, Dino Ferrero Merlino, Wolfgang Hoschek, Tony Johnson, Andreas Pfeiffer, and others. Check the FreeHEP home page for more info. Permission to use and/or redistribute this work is granted under the terms of the LGPL License, with the exception that any usage related to military applications is expressly forbidden. The software and documentation made available under the terms of this license are provided with no warranty. + + +======================================================================== +Fo SnapTree: +======================================================================== + +SNAPTREE LICENSE + +Copyright (c) 2009-2012 Stanford University, unless otherwise specified. +All rights reserved. + +This software was developed by the Pervasive Parallelism Laboratory of +Stanford University, California, USA. + +Permission to use, copy, modify, and distribute this software in source +or binary form for any purpose with or without fee is hereby granted, +provided that the following conditions are met: + + 1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + 2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + + 3. Neither the name of Stanford University nor the names of its + contributors may be used to endorse or promote products derived + from this software without specific prior written permission. + + +THIS SOFTWARE IS PROVIDED BY THE REGENTS AND CONTRIBUTORS ``AS IS'' AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE REGENTS OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT +LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY +OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF +SUCH DAMAGE. + + +======================================================================== +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 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) + (BSD licence) ANTLR ST4 4.0.4 (org.antlr:ST4:4.0.4 - http://www.stringtemplate.org) + (BSD licence) ANTLR StringTemplate (org.antlr:stringtemplate:3.2.1 - http://www.stringtemplate.org) + (BSD style) Hamcrest Core (org.hamcrest:hamcrest-core:1.1 - no url defined) + (BSD) JLine (jline:jline:0.9.94 - http://jline.sourceforge.net) + (BSD) ParaNamer Core (com.thoughtworks.paranamer:paranamer:2.3 - http://paranamer.codehaus.org/paranamer) + (BSD) ParaNamer Core (com.thoughtworks.paranamer:paranamer:2.6 - http://paranamer.codehaus.org/paranamer) + (BSD-like) (The BSD License) jline (org.scala-lang:jline:2.10.4 - http://www.scala-lang.org/) + (BSD-like) Scala Actors library (org.scala-lang:scala-actors:2.10.4 - http://www.scala-lang.org/) + (BSD-like) Scala Compiler (org.scala-lang:scala-compiler:2.10.4 - http://www.scala-lang.org/) + (BSD-like) Scala Compiler (org.scala-lang:scala-reflect:2.10.4 - http://www.scala-lang.org/) + (BSD-like) Scala Library (org.scala-lang:scala-library:2.10.4 - http://www.scala-lang.org/) + (BSD-like) Scalap (org.scala-lang:scalap:2.10.4 - http://www.scala-lang.org/) + (BSD-style) scalacheck (org.scalacheck:scalacheck_2.10:1.10.0 - http://www.scalacheck.org) + (BSD-style) spire (org.spire-math:spire_2.10:0.7.1 - http://spire-math.org) + (BSD-style) spire-macros (org.spire-math:spire-macros_2.10:0.7.1 - http://spire-math.org) + (New BSD License) Kryo (com.esotericsoftware.kryo:kryo:2.21 - http://code.google.com/p/kryo/) + (New BSD License) MinLog (com.esotericsoftware.minlog:minlog:1.2 - http://code.google.com/p/minlog/) + (New BSD License) ReflectASM (com.esotericsoftware.reflectasm:reflectasm:1.07 - http://code.google.com/p/reflectasm/) + (New BSD license) Protocol Buffer Java API (com.google.protobuf:protobuf-java:2.5.0 - http://code.google.com/p/protobuf) + (New BSD license) Protocol Buffer Java API (org.spark-project.protobuf:protobuf-java:2.4.1-shaded - http://code.google.com/p/protobuf) + (The BSD License) Fortran to Java ARPACK (net.sourceforge.f2j:arpack_combined_all:0.1 - http://f2j.sourceforge.net) + (The BSD License) xmlenc Library (xmlenc:xmlenc:0.52 - http://xmlenc.sourceforge.net) + (The New BSD License) Py4J (net.sf.py4j:py4j:0.8.1 - http://py4j.sourceforge.net/) + (Two-clause BSD-style license) JUnit-Interface (com.novocode:junit-interface:0.10 - http://github.com/szeiger/junit-interface/) + (ISC/BSD License) jbcrypt (org.mindrot:jbcrypt:0.3m - http://www.mindrot.org/) + +======================================================================== +MIT licenses +======================================================================== + +The following components are provided under the MIT License. See project link for details. + + (MIT License) JCL 1.1.1 implemented over SLF4J (org.slf4j:jcl-over-slf4j:1.7.5 - http://www.slf4j.org) + (MIT License) JUL to SLF4J bridge (org.slf4j:jul-to-slf4j:1.7.5 - http://www.slf4j.org) + (MIT License) SLF4J API Module (org.slf4j:slf4j-api:1.7.5 - http://www.slf4j.org) + (MIT License) SLF4J LOG4J-12 Binding (org.slf4j:slf4j-log4j12:1.7.5 - http://www.slf4j.org) + (MIT License) pyrolite (org.spark-project:pyrolite:2.0.1 - http://pythonhosted.org/Pyro4/) + (MIT License) scopt (com.github.scopt:scopt_2.10:3.2.0 - https://github.com/scopt/scopt) + (The MIT License) Mockito (org.mockito:mockito-all:1.8.5 - http://www.mockito.org) diff --git a/NOTICE b/NOTICE index 42f6c3a835725..452aef2871652 100644 --- a/NOTICE +++ b/NOTICE @@ -4,11 +4,571 @@ Copyright 2014 The Apache Software Foundation. This product includes software developed at The Apache Software Foundation (http://www.apache.org/). -In addition, this product includes: -- JUnit (http://www.junit.org) is a testing framework for Java. We included it - under the terms of the Eclipse Public License v1.0. +======================================================================== +Common Development and Distribution License 1.0 +======================================================================== -- JTransforms (https://sites.google.com/site/piotrwendykier/software/jtransforms) - provides fast transforms in Java. It is tri-licensed, and we included it under - the terms of the Mozilla Public License v1.1. +The following components are provided under the Common Development and Distribution License 1.0. See project link for details. + + (CDDL 1.0) Glassfish Jasper (org.mortbay.jetty:jsp-2.1:6.1.14 - http://jetty.mortbay.org/project/modules/jsp-2.1) + (CDDL 1.0) Servlet Specification 2.5 API (org.mortbay.jetty:servlet-api-2.5:6.1.14 - http://jetty.mortbay.org/project/modules/servlet-api-2.5) + (COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0) (GNU General Public Library) Streaming API for XML (javax.xml.stream:stax-api:1.0-2 - no url defined) + (Common Development and Distribution License (CDDL) v1.0) JavaBeans Activation Framework (JAF) (javax.activation:activation:1.1 - http://java.sun.com/products/javabeans/jaf/index.jsp) + +======================================================================== +Common Development and Distribution License 1.1 +======================================================================== + +The following components are provided under the Common Development and Distribution License 1.1. See project link for details. + + (CDDL 1.1) (GPL2 w/ CPE) JAXB API bundle for GlassFish V3 (javax.xml.bind:jaxb-api:2.2.2 - https://jaxb.dev.java.net/) + (CDDL 1.1) (GPL2 w/ CPE) JAXB RI (com.sun.xml.bind:jaxb-impl:2.2.3-1 - http://jaxb.java.net/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:1.8 - https://jersey.dev.java.net/jersey-core/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:1.9 - https://jersey.java.net/jersey-core/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-guice (com.sun.jersey.contribs:jersey-guice:1.9 - https://jersey.java.net/jersey-contribs/jersey-guice/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-json (com.sun.jersey:jersey-json:1.8 - https://jersey.dev.java.net/jersey-json/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-json (com.sun.jersey:jersey-json:1.9 - https://jersey.java.net/jersey-json/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:1.8 - https://jersey.dev.java.net/jersey-server/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:1.9 - https://jersey.java.net/jersey-server/) + +======================================================================== +Common Public License 1.0 +======================================================================== + +The following components are provided under the Common Public 1.0 License. See project link for details. + + (Common Public License Version 1.0) JUnit (junit:junit-dep:4.10 - http://junit.org) + (Common Public License Version 1.0) JUnit (junit:junit:3.8.1 - http://junit.org) + (Common Public License Version 1.0) JUnit (junit:junit:4.8.2 - http://junit.org) + +======================================================================== +Eclipse Public License 1.0 +======================================================================== + +The following components are provided under the Eclipse Public License 1.0. See project link for details. + + (Eclipse Public License - Version 1.0) mqtt-client (org.eclipse.paho:mqtt-client:0.4.0 - http://www.eclipse.org/paho/mqtt-client) + (Eclipse Public License v1.0) Eclipse JDT Core (org.eclipse.jdt:core:3.1.1 - http://www.eclipse.org/jdt/) + +======================================================================== +Mozilla Public License 1.0 +======================================================================== + +The following components are provided under the Mozilla Public License 1.0. See project link for details. + + (GPL) (LGPL) (MPL) JTransforms (com.github.rwl:jtransforms:2.4.0 - http://sourceforge.net/projects/jtransforms/) + (Mozilla Public License Version 1.1) jamon-runtime (org.jamon:jamon-runtime:2.3.1 - http://www.jamon.org/jamon-runtime/) + + + +======================================================================== +NOTICE files +======================================================================== + +The following NOTICEs are pertain to software distributed with this project. + + +// ------------------------------------------------------------------ +// NOTICE file corresponding to the section 4d of The Apache License, +// Version 2.0, in this case for +// ------------------------------------------------------------------ + +Apache Avro +Copyright 2009-2013 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +Apache Commons Codec +Copyright 2002-2009 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + +-------------------------------------------------------------------------------- +src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java contains +test data from http://aspell.sourceforge.net/test/batch0.tab. + +Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org). Verbatim copying +and distribution of this entire article is permitted in any medium, +provided this notice is preserved. +-------------------------------------------------------------------------------- + +Apache HttpComponents HttpClient +Copyright 1999-2011 The Apache Software Foundation + +This project contains annotations derived from JCIP-ANNOTATIONS +Copyright (c) 2005 Brian Goetz and Tim Peierls. See http://www.jcip.net + +Apache HttpComponents HttpCore +Copyright 2005-2011 The Apache Software Foundation + +Curator Recipes +Copyright 2011-2014 The Apache Software Foundation + +Curator Framework +Copyright 2011-2014 The Apache Software Foundation + +Curator Client +Copyright 2011-2014 The Apache Software Foundation + +Apache Geronimo +Copyright 2003-2008 The Apache Software Foundation + +Activation 1.1 +Copyright 2003-2007 The Apache Software Foundation + +Apache Commons Lang +Copyright 2001-2014 The Apache Software Foundation + +This product includes software from the Spring Framework, +under the Apache License 2.0 (see: StringUtils.containsWhitespace()) + +Apache log4j +Copyright 2007 The Apache Software Foundation + +# Compress LZF + +This library contains efficient implementation of LZF compression format, +as well as additional helper classes that build on JDK-provided gzip (deflat) +codec. + +## Licensing + +Library is licensed under Apache License 2.0, as per accompanying LICENSE file. + +## Credit + +Library has been written by Tatu Saloranta (tatu.saloranta@iki.fi). +It was started at Ning, inc., as an official Open Source process used by +platform backend, but after initial versions has been developed outside of +Ning by supporting community. + +Other contributors include: + +* Jon Hartlaub (first versions of streaming reader/writer; unit tests) +* Cedrik Lime: parallel LZF implementation + +Various community members have contributed bug reports, and suggested minor +fixes; these can be found from file "VERSION.txt" in SCM. + +Objenesis +Copyright 2006-2009 Joe Walnes, Henri Tremblay, Leonardo Mesquita + +Apache Commons Net +Copyright 2001-2010 The Apache Software Foundation + + The Netty Project + ================= + +Please visit the Netty web site for more information: + + * http://netty.io/ + +Copyright 2011 The Netty Project + +The Netty Project licenses this file to you under the Apache License, +version 2.0 (the "License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at: + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, WITHOUT +WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the +License for the specific language governing permissions and limitations +under the License. + +Also, please refer to each LICENSE..txt file, which is located in +the 'license' directory of the distribution file, for the license terms of the +components that this product depends on. + +------------------------------------------------------------------------------- +This product contains the extensions to Java Collections Framework which has +been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: + + * LICENSE: + * license/LICENSE.jsr166y.txt (Public Domain) + * HOMEPAGE: + * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ + * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ + +This product contains a modified version of Robert Harder's Public Domain +Base64 Encoder and Decoder, which can be obtained at: + + * LICENSE: + * license/LICENSE.base64.txt (Public Domain) + * HOMEPAGE: + * http://iharder.sourceforge.net/current/java/base64/ + +This product contains a modified version of 'JZlib', a re-implementation of +zlib in pure Java, which can be obtained at: + + * LICENSE: + * license/LICENSE.jzlib.txt (BSD Style License) + * HOMEPAGE: + * http://www.jcraft.com/jzlib/ + +This product optionally depends on 'Protocol Buffers', Google's data +interchange format, which can be obtained at: + + * LICENSE: + * license/LICENSE.protobuf.txt (New BSD License) + * HOMEPAGE: + * http://code.google.com/p/protobuf/ + +This product optionally depends on 'SLF4J', a simple logging facade for Java, +which can be obtained at: + + * LICENSE: + * license/LICENSE.slf4j.txt (MIT License) + * HOMEPAGE: + * http://www.slf4j.org/ + +This product optionally depends on 'Apache Commons Logging', a logging +framework, which can be obtained at: + + * LICENSE: + * license/LICENSE.commons-logging.txt (Apache License 2.0) + * HOMEPAGE: + * http://commons.apache.org/logging/ + +This product optionally depends on 'Apache Log4J', a logging framework, +which can be obtained at: + + * LICENSE: + * license/LICENSE.log4j.txt (Apache License 2.0) + * HOMEPAGE: + * http://logging.apache.org/log4j/ + +This product optionally depends on 'JBoss Logging', a logging framework, +which can be obtained at: + + * LICENSE: + * license/LICENSE.jboss-logging.txt (GNU LGPL 2.1) + * HOMEPAGE: + * http://anonsvn.jboss.org/repos/common/common-logging-spi/ + +This product optionally depends on 'Apache Felix', an open source OSGi +framework implementation, which can be obtained at: + + * LICENSE: + * license/LICENSE.felix.txt (Apache License 2.0) + * HOMEPAGE: + * http://felix.apache.org/ + +This product optionally depends on 'Webbit', a Java event based +WebSocket and HTTP server: + + * LICENSE: + * license/LICENSE.webbit.txt (BSD License) + * HOMEPAGE: + * https://github.com/joewalnes/webbit + +# Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +Jackson core and extension components may be licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. + +Jackson core and extension components may licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +mesos +Copyright 2014 The Apache Software Foundation + +Apache Thrift +Copyright 2006-2010 The Apache Software Foundation. + + Apache Ant + Copyright 1999-2013 The Apache Software Foundation + + The task is based on code Copyright (c) 2002, Landmark + Graphics Corp that has been kindly donated to the Apache Software + Foundation. + +Apache Commons IO +Copyright 2002-2012 The Apache Software Foundation + +Apache Commons Math +Copyright 2001-2013 The Apache Software Foundation + +=============================================================================== + +The inverse error function implementation in the Erf class is based on CUDA +code developed by Mike Giles, Oxford-Man Institute of Quantitative Finance, +and published in GPU Computing Gems, volume 2, 2010. +=============================================================================== + +The BracketFinder (package org.apache.commons.math3.optimization.univariate) +and PowellOptimizer (package org.apache.commons.math3.optimization.general) +classes are based on the Python code in module "optimize.py" (version 0.5) +developed by Travis E. Oliphant for the SciPy library (http://www.scipy.org/) +Copyright © 2003-2009 SciPy Developers. +=============================================================================== + +The LinearConstraint, LinearObjectiveFunction, LinearOptimizer, +RelationShip, SimplexSolver and SimplexTableau classes in package +org.apache.commons.math3.optimization.linear include software developed by +Benjamin McCann (http://www.benmccann.com) and distributed with +the following copyright: Copyright 2009 Google Inc. +=============================================================================== + +This product includes software developed by the +University of Chicago, as Operator of Argonne National +Laboratory. +The LevenbergMarquardtOptimizer class in package +org.apache.commons.math3.optimization.general includes software +translated from the lmder, lmpar and qrsolv Fortran routines +from the Minpack package +Minpack Copyright Notice (1999) University of Chicago. All rights reserved +=============================================================================== + +The GraggBulirschStoerIntegrator class in package +org.apache.commons.math3.ode.nonstiff includes software translated +from the odex Fortran routine developed by E. Hairer and G. Wanner. +Original source copyright: +Copyright (c) 2004, Ernst Hairer +=============================================================================== + +The EigenDecompositionImpl class in package +org.apache.commons.math3.linear includes software translated +from some LAPACK Fortran routines. Original source copyright: +Copyright (c) 1992-2008 The University of Tennessee. All rights reserved. +=============================================================================== + +The MersenneTwister class in package org.apache.commons.math3.random +includes software translated from the 2002-01-26 version of +the Mersenne-Twister generator written in C by Makoto Matsumoto and Takuji +Nishimura. Original source copyright: +Copyright (C) 1997 - 2002, Makoto Matsumoto and Takuji Nishimura, +All rights reserved +=============================================================================== + +The LocalizedFormatsTest class in the unit tests is an adapted version of +the OrekitMessagesTest class from the orekit library distributed under the +terms of the Apache 2 licence. Original source copyright: +Copyright 2010 CS Systèmes d'Information +=============================================================================== + +The HermiteInterpolator class and its corresponding test have been imported from +the orekit library distributed under the terms of the Apache 2 licence. Original +source copyright: +Copyright 2010-2012 CS Systèmes d'Information +=============================================================================== + +The creation of the package "o.a.c.m.analysis.integration.gauss" was inspired +by an original code donated by Sébastien Brisard. +=============================================================================== + +The complete text of licenses and disclaimers associated with the the original +sources enumerated above at the time of code translation are in the LICENSE.txt +file. + +This product currently only contains code developed by authors +of specific components, as identified by the source code files; +if such notes are missing files have been created by +Tatu Saloranta. + +For additional credits (generally to people who reported problems) +see CREDITS file. + +Apache Commons Lang +Copyright 2001-2011 The Apache Software Foundation + +Apache Commons Compress +Copyright 2002-2012 The Apache Software Foundation + +Apache Commons CLI +Copyright 2001-2009 The Apache Software Foundation + +Google Guice - Extensions - Servlet +Copyright 2006-2011 Google, Inc. + +Google Guice - Core Library +Copyright 2006-2011 Google, Inc. + +Apache Jakarta HttpClient +Copyright 1999-2007 The Apache Software Foundation + +Apache Hive +Copyright 2008-2013 The Apache Software Foundation + +This product includes software developed by The Apache Software +Foundation (http://www.apache.org/). + +This product includes software developed by The JDBM Project +(http://jdbm.sourceforge.net/). + +This product includes/uses ANTLR (http://www.antlr.org/), +Copyright (c) 2003-2011, Terrence Parr. + +This product includes/uses StringTemplate (http://www.stringtemplate.org/), +Copyright (c) 2011, Terrence Parr. + +This product includes/uses ASM (http://asm.ow2.org/), +Copyright (c) 2000-2007 INRIA, France Telecom. + +This product includes/uses org.json (http://www.json.org/java/index.html), +Copyright (c) 2002 JSON.org + +This product includes/uses JLine (http://jline.sourceforge.net/), +Copyright (c) 2002-2006, Marc Prud'hommeaux . + +This product includes/uses SQLLine (http://sqlline.sourceforge.net), +Copyright (c) 2002, 2003, 2004, 2005 Marc Prud'hommeaux . + +This product includes/uses SLF4J (http://www.slf4j.org/), +Copyright (c) 2004-2010 QOS.ch + +This product includes/uses Bootstrap (http://twitter.github.com/bootstrap/), +Copyright (c) 2012 Twitter, Inc. + +This product includes/uses Glyphicons (http://glyphicons.com/), +Copyright (c) 2010 - 2012 Jan Kovarík + +This product includes DataNucleus (http://www.datanucleus.org/) +Copyright 2008-2008 DataNucleus + +This product includes Guava (http://code.google.com/p/guava-libraries/) +Copyright (C) 2006 Google Inc. + +This product includes JavaEWAH (http://code.google.com/p/javaewah/) +Copyright (C) 2011 Google Inc. + +Apache Commons Pool +Copyright 1999-2009 The Apache Software Foundation + +========================================================================= +== NOTICE file corresponding to section 4(d) of the Apache License, == +== Version 2.0, in this case for the DataNucleus distribution. == +========================================================================= + +=================================================================== +This product includes software developed by many individuals, +including the following: +=================================================================== +Erik Bengtson +Andy Jefferson + +=================================================================== +This product has included contributions from some individuals, +including the following: +=================================================================== + +=================================================================== +This product has included contributions from some individuals, +including the following: +=================================================================== +Joerg von Frantzius +Thomas Marti +Barry Haddow +Marco Schulze +Ralph Ullrich +David Ezzio +Brendan de Beer +David Eaves +Martin Taal +Tony Lai +Roland Szabo +Marcus Mennemeier +Xuan Baldauf +Eric Sultan + +=================================================================== +This product also includes software developed by the TJDO project +(http://tjdo.sourceforge.net/). +=================================================================== + +=================================================================== +This product includes software developed by many individuals, +including the following: +=================================================================== +Andy Jefferson +Erik Bengtson +Joerg von Frantzius +Marco Schulze + +=================================================================== +This product has included contributions from some individuals, +including the following: +=================================================================== +Barry Haddow +Ralph Ullrich +David Ezzio +Brendan de Beer +David Eaves +Martin Taal +Tony Lai +Roland Szabo +Anton Troshin (Timesten) + +=================================================================== +This product also includes software developed by the Apache Commons project +(http://commons.apache.org/). +=================================================================== + +Apache Java Data Objects (JDO) +Copyright 2005-2006 The Apache Software Foundation + +========================================================================= +== NOTICE file corresponding to section 4(d) of the Apache License, == +== Version 2.0, in this case for the Apache Derby distribution. == +========================================================================= + +Apache Derby +Copyright 2004-2008 The Apache Software Foundation + +Portions of Derby were originally developed by +International Business Machines Corporation and are +licensed to the Apache Software Foundation under the +"Software Grant and Corporate Contribution License Agreement", +informally known as the "Derby CLA". +The following copyright notice(s) were affixed to portions of the code +with which this file is now or was at one time distributed +and are placed here unaltered. + +(C) Copyright 1997,2004 International Business Machines Corporation. All rights reserved. + +(C) Copyright IBM Corp. 2003. + +The portion of the functionTests under 'nist' was originally +developed by the National Institute of Standards and Technology (NIST), +an agency of the United States Department of Commerce, and adapted by +International Business Machines Corporation in accordance with the NIST +Software Acknowledgment and Redistribution document at +http://www.itl.nist.gov/div897/ctg/sql_form.htm + +Apache Commons Collections +Copyright 2001-2008 The Apache Software Foundation + +Apache Commons Configuration +Copyright 2001-2008 The Apache Software Foundation + +Apache Jakarta Commons Digester +Copyright 2001-2006 The Apache Software Foundation + +Apache Commons BeanUtils +Copyright 2000-2008 The Apache Software Foundation + +Apache Avro Mapred API +Copyright 2009-2013 The Apache Software Foundation + +Apache Avro IPC +Copyright 2009-2013 The Apache Software Foundation diff --git a/README.md b/README.md index dc8135b9b8b51..6211a5889a3f5 100644 --- a/README.md +++ b/README.md @@ -9,42 +9,61 @@ You can find the latest Spark documentation, including a programming guide, on the project webpage at . This README file only contains basic setup instructions. +## Building Spark -## Building - -Spark requires Scala 2.10. The project is built using Simple Build Tool (SBT), -which can be obtained [here](http://www.scala-sbt.org). If SBT is installed we -will use the system version of sbt otherwise we will attempt to download it -automatically. To build Spark and its example programs, run: +Spark is built on Scala 2.10. To build Spark and its example programs, run: ./sbt/sbt assembly -Once you've built Spark, the easiest way to start using it is the shell: +(You do not need to do this if you downloaded a pre-built package.) + +## Interactive Scala Shell + +The easiest way to start using Spark is through the Scala shell: ./bin/spark-shell -Or, for the Python API, the Python shell (`./bin/pyspark`). +Try the following command, which should return 1000: + + scala> sc.parallelize(1 to 1000).count() + +## Interactive Python Shell + +Alternatively, if you prefer Python, you can use the Python shell: + + ./bin/pyspark + +And run the following command, which should also return 1000: + + >>> sc.parallelize(range(1000)).count() + +## Example Programs Spark also comes with several sample programs in the `examples` directory. -To run one of them, use `./bin/run-example `. For example: +To run one of them, use `./bin/run-example [params]`. For example: - ./bin/run-example org.apache.spark.examples.SparkLR local[2] + ./bin/run-example SparkPi -will run the Logistic Regression example locally on 2 CPUs. +will run the Pi example locally. -Each of the example programs prints usage help if no params are given. +You can set the MASTER environment variable when running examples to submit +examples to a cluster. This can be a mesos:// or spark:// URL, +"yarn-cluster" or "yarn-client" to run on YARN, and "local" to run +locally with one thread, or "local[N]" to run locally with N threads. You +can also use an abbreviated class name if the class is in the `examples` +package. For instance: -All of the Spark samples take a `` parameter that is the cluster URL -to connect to. This can be a mesos:// or spark:// URL, or "local" to run -locally with one thread, or "local[N]" to run locally with N threads. + MASTER=spark://host:7077 ./bin/run-example SparkPi -## Running tests +Many of the example programs print usage help if no params are given. -Testing first requires [Building](#building) Spark. Once Spark is built, tests +## Running Tests + +Testing first requires [building Spark](#building-spark). Once Spark is built, tests can be run using: -`./sbt/sbt test` - + ./sbt/sbt test + ## A Note About Hadoop Versions Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported diff --git a/assembly/pom.xml b/assembly/pom.xml index b5e752c6cd1f6..0c60b66c3daca 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml @@ -33,21 +33,13 @@ scala-${scala.binary.version} - ${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar + spark-assembly-${project.version}-hadoop${hadoop.version}.jar ${project.build.directory}/${spark.jar.dir}/${spark.jar.basename} spark /usr/share/spark root - - - - lib - file://${project.basedir}/lib - - - org.apache.spark @@ -84,11 +76,6 @@ spark-sql_${scala.binary.version} ${project.version} - - net.sf.py4j - py4j - 0.8.1 - @@ -109,6 +96,7 @@ *:* + org/datanucleus/** META-INF/*.SF META-INF/*.DSA META-INF/*.RSA @@ -134,6 +122,8 @@ log4j.properties + + @@ -163,6 +153,16 @@ + + hive + + + org.apache.spark + spark-hive_${scala.binary.version} + ${project.version} + + + spark-ganglia-lgpl @@ -208,7 +208,7 @@ org.codehaus.mojo buildnumber-maven-plugin - 1.1 + 1.2 validate diff --git a/bagel/pom.xml b/bagel/pom.xml index 142f75c5d2c64..c8e39a415af28 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml @@ -31,20 +31,6 @@ Spark Project Bagel http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.spark diff --git a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala index 70c7474a936dc..70a99b33d753c 100644 --- a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala @@ -220,20 +220,23 @@ object Bagel extends Logging { */ private def comp[K: Manifest, V <: Vertex, M <: Message[K], C]( sc: SparkContext, - grouped: RDD[(K, (Seq[C], Seq[V]))], + grouped: RDD[(K, (Iterable[C], Iterable[V]))], compute: (V, Option[C]) => (V, Array[M]), storageLevel: StorageLevel ): (RDD[(K, (V, Array[M]))], Int, Int) = { var numMsgs = sc.accumulator(0) var numActiveVerts = sc.accumulator(0) - val processed = grouped.flatMapValues { - case (_, vs) if vs.size == 0 => None - case (c, vs) => + val processed = grouped.mapValues(x => (x._1.iterator, x._2.iterator)) + .flatMapValues { + case (_, vs) if !vs.hasNext => None + case (c, vs) => { val (newVert, newMsgs) = - compute(vs(0), c match { - case Seq(comb) => Some(comb) - case Seq() => None - }) + compute(vs.next, + c.hasNext match { + case true => Some(c.next) + case false => None + } + ) numMsgs += newMsgs.size if (newVert.active) { @@ -241,6 +244,7 @@ object Bagel extends Logging { } Some((newVert, newMsgs)) + } }.persist(storageLevel) // Force evaluation of processed RDD for accurate performance measurements diff --git a/bagel/src/main/scala/org/apache/spark/bagel/package-info.java b/bagel/src/main/scala/org/apache/spark/bagel/package-info.java new file mode 100644 index 0000000000000..81f26f276549f --- /dev/null +++ b/bagel/src/main/scala/org/apache/spark/bagel/package-info.java @@ -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. + */ + +/** + * Bagel: An implementation of Pregel in Spark. THIS IS DEPRECATED - use Spark's GraphX library. + */ +package org.apache.spark.bagel; \ No newline at end of file diff --git a/bagel/src/main/scala/org/apache/spark/bagel/package.scala b/bagel/src/main/scala/org/apache/spark/bagel/package.scala new file mode 100644 index 0000000000000..2fb1934579781 --- /dev/null +++ b/bagel/src/main/scala/org/apache/spark/bagel/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Bagel: An implementation of Pregel in Spark. THIS IS DEPRECATED - use Spark's GraphX library. + */ +package object bagel diff --git a/bagel/src/test/resources/log4j.properties b/bagel/src/test/resources/log4j.properties index 5cdcf35b23a6c..30b4baa4d714a 100644 --- a/bagel/src/test/resources/log4j.properties +++ b/bagel/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=bagel/target/unit-tests.log +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} %p %c{1}: %m%n diff --git a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala index 9c37fadb78d2f..55241d33cd3f0 100644 --- a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala +++ b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala @@ -24,20 +24,20 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.storage.StorageLevel +import scala.language.postfixOps + class TestVertex(val active: Boolean, val age: Int) extends Vertex with Serializable class TestMessage(val targetId: String) extends Message[String] with Serializable class BagelSuite extends FunSuite with Assertions with BeforeAndAfter with Timeouts { - + var sc: SparkContext = _ - + after { if (sc != null) { sc.stop() sc = null } - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.driver.port") } test("halting by voting") { @@ -80,7 +80,7 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter with Timeo test("large number of iterations") { // This tests whether jobs with a large number of iterations finish in a reasonable time, // because non-memoized recursion in RDD or DAGScheduler used to cause them to hang - failAfter(10 seconds) { + failAfter(30 seconds) { sc = new SparkContext("local", "test") val verts = sc.parallelize((1 to 4).map(id => (id.toString, new TestVertex(true, 0)))) val msgs = sc.parallelize(Array[(String, TestMessage)]()) @@ -101,7 +101,7 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter with Timeo sc = new SparkContext("local", "test") val verts = sc.parallelize((1 to 4).map(id => (id.toString, new TestVertex(true, 0)))) val msgs = sc.parallelize(Array[(String, TestMessage)]()) - val numSupersteps = 50 + val numSupersteps = 20 val result = Bagel.run(sc, verts, msgs, sc.defaultParallelism, StorageLevel.DISK_ONLY) { (self: TestVertex, msgs: Option[Array[TestMessage]], superstep: Int) => diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 4f60bff19cb93..58710cd1bd548 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -1,69 +1,110 @@ -@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 computes Spark's classpath and prints it to stdout; it's used by both the "run" -rem script and the ExecutorRunner in standalone cluster mode. - -set SCALA_VERSION=2.10 - -rem Figure out where the Spark framework is installed -set FWDIR=%~dp0..\ - -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" - -rem Build up classpath -set CLASSPATH=%FWDIR%conf -if exist "%FWDIR%RELEASE" ( - for %%d in ("%FWDIR%jars\spark-assembly*.jar") do ( - set ASSEMBLY_JAR=%%d - ) -) else ( - for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( - set ASSEMBLY_JAR=%%d - ) -) -set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR% - -if "x%SPARK_TESTING%"=="x1" ( - rem Add test clases to path - set CLASSPATH=%CLASSPATH%;%FWDIR%core\target\scala-%SCALA_VERSION%\test-classes - set CLASSPATH=%CLASSPATH%;%FWDIR%repl\target\scala-%SCALA_VERSION%\test-classes - set CLASSPATH=%CLASSPATH%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\test-classes - set CLASSPATH=%CLASSPATH%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\test-classes - set CLASSPATH=%CLASSPATH%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\test-classes -) - -rem Add hadoop conf dir - else FileSystem.*, etc fail -rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts -rem the configurtion files. -if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir - set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR% -:no_hadoop_conf_dir - -if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir - set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% -:no_yarn_conf_dir - -rem A bit of a hack to allow calling this script within run2.cmd without seeing output -if "%DONT_PRINT_CLASSPATH%"=="1" goto exit - -echo %CLASSPATH% - -:exit +@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 computes Spark's classpath and prints it to stdout; it's used by both the "run" +rem script and the ExecutorRunner in standalone cluster mode. + +rem If we're called from spark-class2.cmd, it already set enabledelayedexpansion and setting +rem it here would stop us from affecting its copy of the CLASSPATH variable; otherwise we +rem need to set it here because we use !datanucleus_jars! below. +if "%DONT_PRINT_CLASSPATH%"=="1" goto skip_delayed_expansion +setlocal enabledelayedexpansion +:skip_delayed_expansion + +set SCALA_VERSION=2.10 + +rem Figure out where the Spark framework is installed +set FWDIR=%~dp0..\ + +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" + +rem Build up classpath +set CLASSPATH=%FWDIR%conf +if exist "%FWDIR%RELEASE" ( + for %%d in ("%FWDIR%lib\spark-assembly*.jar") do ( + set ASSEMBLY_JAR=%%d + ) +) else ( + for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( + set ASSEMBLY_JAR=%%d + ) +) + +set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR% + +rem When Hive support is needed, Datanucleus jars must be included on the classpath. +rem Datanucleus jars do not work if only included in the uber jar as plugin.xml metadata is lost. +rem Both sbt and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is +rem built with Hive, so look for them there. +if exist "%FWDIR%RELEASE" ( + set datanucleus_dir=%FWDIR%lib +) else ( + set datanucleus_dir=%FWDIR%lib_managed\jars +) +set "datanucleus_jars=" +for %%d in ("%datanucleus_dir%\datanucleus-*.jar") do ( + set datanucleus_jars=!datanucleus_jars!;%%d +) +set CLASSPATH=%CLASSPATH%;%datanucleus_jars% + +set SPARK_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%graphx\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%tools\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\catalyst\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\core\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\hive\target\scala-%SCALA_VERSION%\classes + +set SPARK_TEST_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%graphx\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\catalyst\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\core\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\hive\target\scala-%SCALA_VERSION%\test-classes + +if "x%SPARK_TESTING%"=="x1" ( + rem Add test clases to path - note, add SPARK_CLASSES and SPARK_TEST_CLASSES before CLASSPATH + rem so that local compilation takes precedence over assembled jar + set CLASSPATH=%SPARK_CLASSES%;%SPARK_TEST_CLASSES%;%CLASSPATH% +) + +rem Add hadoop conf dir - else FileSystem.*, etc fail +rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts +rem the configurtion files. +if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir + set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR% +:no_hadoop_conf_dir + +if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir + set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% +:no_yarn_conf_dir + +rem A bit of a hack to allow calling this script within run2.cmd without seeing output +if "%DONT_PRINT_CLASSPATH%"=="1" goto exit + +echo %CLASSPATH% + +:exit diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index bef42df71ce01..2cf4e381c1c88 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -28,26 +28,20 @@ FWDIR="$(cd `dirname $0`/..; pwd)" . $FWDIR/bin/load-spark-env.sh # Build up classpath -CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" +CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH:$FWDIR/conf" -# Support for interacting with Hive. Since hive pulls in a lot of dependencies that might break -# existing Spark applications, it is not included in the standard spark assembly. Instead, we only -# include it in the classpath if the user has explicitly requested it by running "sbt hive/assembly" -# Hopefully we will find a way to avoid uber-jars entirely and deploy only the needed packages in -# the future. -if [ -f "$FWDIR"/sql/hive/target/scala-$SCALA_VERSION/spark-hive-assembly-*.jar ]; then +ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION" - # Datanucleus jars do not work if only included in the uberjar as plugin.xml metadata is lost. - DATANUCLEUSJARS=$(JARS=("$FWDIR/lib_managed/jars"/datanucleus-*.jar); IFS=:; echo "${JARS[*]}") - CLASSPATH=$CLASSPATH:$DATANUCLEUSJARS - - ASSEMBLY_DIR="$FWDIR/sql/hive/target/scala-$SCALA_VERSION/" +if [ -n "$JAVA_HOME" ]; then + JAR_CMD="$JAVA_HOME/bin/jar" else - ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION/" + JAR_CMD="jar" fi -# First check if we have a dependencies jar. If so, include binary classes with the deps jar -if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then +# A developer option to prepend more recently compiled Spark classes +if [ -n "$SPARK_PREPEND_CLASSES" ]; then + echo "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark"\ + "classes ahead of assembly." >&2 CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/classes" @@ -58,17 +52,65 @@ if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/yarn/stable/target/scala-$SCALA_VERSION/classes" +fi - DEPS_ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark*-assembly*hadoop*-deps.jar` - CLASSPATH="$CLASSPATH:$DEPS_ASSEMBLY_JAR" +# Use spark-assembly jar from either RELEASE or assembly directory +if [ -f "$FWDIR/RELEASE" ]; then + assembly_folder="$FWDIR"/lib else - # Else use spark-assembly jar from either RELEASE or assembly directory - if [ -f "$FWDIR/RELEASE" ]; then - ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark*-assembly*.jar` - else - ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark*-assembly*hadoop*.jar` + assembly_folder="$ASSEMBLY_DIR" +fi + +num_jars=$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*\.jar" | wc -l) +if [ "$num_jars" -eq "0" ]; then + echo "Failed to find Spark assembly in $assembly_folder" + echo "You need to build Spark before running this program." + exit 1 +fi +if [ "$num_jars" -gt "1" ]; then + jars_list=$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*.jar") + echo "Found multiple Spark assembly jars in $assembly_folder:" + echo "$jars_list" + echo "Please remove all but one jar." + exit 1 +fi + +ASSEMBLY_JAR=$(ls "$assembly_folder"/spark-assembly*hadoop*.jar 2>/dev/null) + +# Verify that versions of java used to build the jars and run Spark are compatible +jar_error_check=$("$JAR_CMD" -tf "$ASSEMBLY_JAR" nonexistent/class/path 2>&1) +if [[ "$jar_error_check" =~ "invalid CEN header" ]]; then + echo "Loading Spark jar with '$JAR_CMD' failed. " + echo "This is likely because Spark was compiled with Java 7 and run " + echo "with Java 6. (see SPARK-1703). Please use Java 7 to run Spark " + echo "or build Spark with Java 6." + exit 1 +fi + +CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" + +# 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. +# Note that this check order is faster (by up to half a second) in the case where Hive is not used. +if [ -f "$FWDIR/RELEASE" ]; then + datanucleus_dir="$FWDIR"/lib +else + datanucleus_dir="$FWDIR"/lib_managed/jars +fi + +datanucleus_jars=$(find "$datanucleus_dir" 2>/dev/null | grep "datanucleus-.*\\.jar") +datanucleus_jars=$(echo "$datanucleus_jars" | tr "\n" : | sed s/:$//g) + +if [ -n "$datanucleus_jars" ]; then + hive_files=$("$JAR_CMD" -tf "$ASSEMBLY_JAR" org/apache/hadoop/hive/ql/exec 2>/dev/null) + if [ -n "$hive_files" ]; then + echo "Spark assembly has been built with Hive, including Datanucleus jars on classpath" 1>&2 + CLASSPATH="$CLASSPATH:$datanucleus_jars" fi - CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" fi # Add test classes if we're running from SBT or Maven with SPARK_TESTING set to 1 @@ -87,10 +129,10 @@ fi # Add hadoop conf dir if given -- otherwise FileSystem.*, etc fail ! # Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts # the configurtion files. -if [ "x" != "x$HADOOP_CONF_DIR" ]; then +if [ -n "$HADOOP_CONF_DIR" ]; then CLASSPATH="$CLASSPATH:$HADOOP_CONF_DIR" fi -if [ "x" != "x$YARN_CONF_DIR" ]; then +if [ -n "$YARN_CONF_DIR" ]; then CLASSPATH="$CLASSPATH:$YARN_CONF_DIR" fi diff --git a/bin/pyspark b/bin/pyspark index cad982bc33477..0b5ed40e2157d 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -17,7 +17,7 @@ # limitations under the License. # -# Figure out where the Scala framework is installed +# Figure out where Spark is installed FWDIR="$(cd `dirname $0`/..; pwd)" # Export this as SPARK_HOME @@ -25,13 +25,19 @@ export SPARK_HOME="$FWDIR" SCALA_VERSION=2.10 +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + echo "Usage: ./bin/pyspark [options]" + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit 0 +fi + # Exit if the user hasn't compiled Spark if [ ! -f "$FWDIR/RELEASE" ]; then # Exit if the user hasn't compiled Spark ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null if [[ $? != 0 ]]; then echo "Failed to find Spark assembly in $FWDIR/assembly/target" >&2 - echo "You need to build Spark with sbt/sbt assembly before running this program" >&2 + echo "You need to build Spark before running this program" >&2 exit 1 fi fi @@ -39,25 +45,57 @@ fi . $FWDIR/bin/load-spark-env.sh # Figure out which Python executable to use -if [ -z "$PYSPARK_PYTHON" ] ; then +if [[ -z "$PYSPARK_PYTHON" ]]; then PYSPARK_PYTHON="python" fi export PYSPARK_PYTHON # Add the PySpark classes to the Python path: export PYTHONPATH=$SPARK_HOME/python/:$PYTHONPATH +export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.1-src.zip:$PYTHONPATH # Load the PySpark shell.py script when ./pyspark is used interactively: export OLD_PYTHONSTARTUP=$PYTHONSTARTUP export PYTHONSTARTUP=$FWDIR/python/pyspark/shell.py -if [ -n "$IPYTHON_OPTS" ]; then +# If IPython options are specified, assume user wants to run IPython +if [[ -n "$IPYTHON_OPTS" ]]; then IPYTHON=1 fi -# Only use ipython if no command line arguments were provided [SPARK-1134] -if [[ "$IPYTHON" = "1" && $# = 0 ]] ; then - exec ipython $IPYTHON_OPTS +# Build up arguments list manually to preserve quotes and backslashes. +# We export Spark submit arguments as an environment variable because shell.py must run as a +# PYTHONSTARTUP script, which does not take in arguments. This is required for IPython notebooks. + +PYSPARK_SUBMIT_ARGS="" +whitespace="[[:space:]]" +for i in "$@"; do + if [[ $i =~ \" ]]; then i=$(echo $i | sed 's/\"/\\\"/g'); fi + if [[ $i =~ $whitespace ]]; then i=\"$i\"; fi + PYSPARK_SUBMIT_ARGS="$PYSPARK_SUBMIT_ARGS $i" +done +export PYSPARK_SUBMIT_ARGS + +# For pyspark tests +if [[ -n "$SPARK_TESTING" ]]; then + if [[ -n "$PYSPARK_DOC_TEST" ]]; then + exec "$PYSPARK_PYTHON" -m doctest $1 + else + exec "$PYSPARK_PYTHON" $1 + fi + exit +fi + +# If a python file is provided, directly run spark-submit. +if [[ "$1" =~ \.py$ ]]; then + echo -e "\nWARNING: Running python applications through ./bin/pyspark is deprecated as of Spark 1.0." 1>&2 + echo -e "Use ./bin/spark-submit \n" 1>&2 + exec $FWDIR/bin/spark-submit "$@" else - exec "$PYSPARK_PYTHON" "$@" + # Only use ipython if no command line arguments were provided [SPARK-1134] + if [[ "$IPYTHON" = "1" ]]; then + exec ipython $IPYTHON_OPTS + else + exec "$PYSPARK_PYTHON" + fi fi diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index 95791095ec932..0ef9eea95342e 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -31,7 +31,7 @@ set FOUND_JAR=0 for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( set FOUND_JAR=1 ) -if "%FOUND_JAR%"=="0" ( +if [%FOUND_JAR%] == [0] ( echo Failed to find Spark assembly JAR. echo You need to build Spark with sbt\sbt assembly before running this program. goto exit @@ -42,14 +42,30 @@ 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" rem Figure out which Python to use. -if "x%PYSPARK_PYTHON%"=="x" set PYSPARK_PYTHON=python +if [%PYSPARK_PYTHON%] == [] set PYSPARK_PYTHON=python set PYTHONPATH=%FWDIR%python;%PYTHONPATH% +set PYTHONPATH=%FWDIR%python\lib\py4j-0.8.1-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%FWDIR%python\pyspark\shell.py +set PYSPARK_SUBMIT_ARGS=%* echo Running %PYSPARK_PYTHON% with PYTHONPATH=%PYTHONPATH% -"%PYSPARK_PYTHON%" %* +rem Check whether the argument is a file +for /f %%i in ('echo %1^| findstr /R "\.py"') do ( + set PYTHON_FILE=%%i +) + +if [%PYTHON_FILE%] == [] ( + %PYSPARK_PYTHON% +) else ( + echo. + echo WARNING: Running python applications through ./bin/pyspark.cmd is deprecated as of Spark 1.0. + echo Use ./bin/spark-submit ^ + echo. + "%FWDIR%\bin\spark-submit.cmd" %PYSPARK_SUBMIT_ARGS% +) + :exit diff --git a/bin/run-example b/bin/run-example index 5af95a08c6c41..e7a5fe3914fbd 100755 --- a/bin/run-example +++ b/bin/run-example @@ -17,76 +17,42 @@ # limitations under the License. # -cygwin=false -case "`uname`" in - CYGWIN*) cygwin=true;; -esac - SCALA_VERSION=2.10 -# Figure out where the Scala framework is installed FWDIR="$(cd `dirname $0`/..; pwd)" - -# Export this as SPARK_HOME export SPARK_HOME="$FWDIR" +EXAMPLES_DIR="$FWDIR"/examples -. $FWDIR/bin/load-spark-env.sh - -if [ -z "$1" ]; then - echo "Usage: run-example []" >&2 +if [ -n "$1" ]; then + EXAMPLE_CLASS="$1" + shift +else + echo "Usage: ./bin/run-example [example-args]" + echo " - set MASTER=XX to use a specific master" + echo " - can use abbreviated example class name (e.g. SparkPi, mllib.LinearRegression)" exit 1 fi -# Figure out the JAR file that our examples were packaged into. This includes a bit of a hack -# to avoid the -sources and -doc packages that are built by publish-local. -EXAMPLES_DIR="$FWDIR"/examples -SPARK_EXAMPLES_JAR="" -if [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar` +if [ -f "$FWDIR/RELEASE" ]; then + export SPARK_EXAMPLES_JAR=`ls "$FWDIR"/lib/spark-examples-*hadoop*.jar` +elif [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar ]; then + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar` fi + if [[ -z $SPARK_EXAMPLES_JAR ]]; then - echo "Failed to find Spark examples assembly in $FWDIR/examples/target" >&2 - echo "You need to build Spark with sbt/sbt assembly before running this program" >&2 + echo "Failed to find Spark examples assembly in $FWDIR/lib or $FWDIR/examples/target" >&2 + echo "You need to build Spark before running this program" >&2 exit 1 fi +EXAMPLE_MASTER=${MASTER:-"local[*]"} -# Since the examples JAR ideally shouldn't include spark-core (that dependency should be -# "provided"), also add our standard Spark classpath, built using compute-classpath.sh. -CLASSPATH=`$FWDIR/bin/compute-classpath.sh` -CLASSPATH="$SPARK_EXAMPLES_JAR:$CLASSPATH" - -if $cygwin; then - CLASSPATH=`cygpath -wp $CLASSPATH` - export SPARK_EXAMPLES_JAR=`cygpath -w $SPARK_EXAMPLES_JAR` -fi - -# Find java binary -if [ -n "${JAVA_HOME}" ]; then - RUNNER="${JAVA_HOME}/bin/java" -else - if [ `command -v java` ]; then - RUNNER="java" - else - echo "JAVA_HOME is not set" >&2 - exit 1 - fi -fi - -# Set JAVA_OPTS to be able to load native libraries and to set heap size -JAVA_OPTS="$SPARK_JAVA_OPTS" -JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH" -# Load extra JAVA_OPTS from conf/java-opts, if it exists -if [ -e "$FWDIR/conf/java-opts" ] ; then - JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`" -fi -export JAVA_OPTS - -if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then - echo -n "Spark Command: " - echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" - echo "========================================" - echo +if [[ ! $EXAMPLE_CLASS == org.apache.spark.examples* ]]; then + EXAMPLE_CLASS="org.apache.spark.examples.$EXAMPLE_CLASS" fi -exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" +"$FWDIR"/bin/spark-submit \ + --master $EXAMPLE_MASTER \ + --class $EXAMPLE_CLASS \ + "$SPARK_EXAMPLES_JAR" \ + "$@" diff --git a/bin/run-example2.cmd b/bin/run-example2.cmd index 40abb9af74246..eadedd7fa61ff 100644 --- a/bin/run-example2.cmd +++ b/bin/run-example2.cmd @@ -30,7 +30,9 @@ if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" rem Test that an argument was given if not "x%1"=="x" goto arg_given - echo Usage: run-example ^ [^] + echo Usage: run-example ^ [example-args] + echo - set MASTER=XX to use a specific master + echo - can use abbreviated example class name (e.g. SparkPi, mllib.LinearRegression) goto exit :arg_given @@ -38,8 +40,14 @@ set EXAMPLES_DIR=%FWDIR%examples rem Figure out the JAR file that our examples were packaged into. set SPARK_EXAMPLES_JAR= -for %%d in ("%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\spark-examples*assembly*.jar") do ( - set SPARK_EXAMPLES_JAR=%%d +if exist "%FWDIR%RELEASE" ( + for %%d in ("%FWDIR%lib\spark-examples*.jar") do ( + set SPARK_EXAMPLES_JAR=%%d + ) +) else ( + for %%d in ("%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\spark-examples*.jar") do ( + set SPARK_EXAMPLES_JAR=%%d + ) ) if "x%SPARK_EXAMPLES_JAR%"=="x" ( echo Failed to find Spark examples assembly JAR. @@ -47,15 +55,34 @@ if "x%SPARK_EXAMPLES_JAR%"=="x" ( goto exit ) -rem Compute Spark classpath using external script -set DONT_PRINT_CLASSPATH=1 -call "%FWDIR%bin\compute-classpath.cmd" -set DONT_PRINT_CLASSPATH=0 -set CLASSPATH=%SPARK_EXAMPLES_JAR%;%CLASSPATH% +rem Set master from MASTER environment variable if given +if "x%MASTER%"=="x" ( + set EXAMPLE_MASTER=local[*] +) else ( + set EXAMPLE_MASTER=%MASTER% +) + +rem If the EXAMPLE_CLASS does not start with org.apache.spark.examples, add that +set EXAMPLE_CLASS=%1 +set PREFIX=%EXAMPLE_CLASS:~0,25% +if not %PREFIX%==org.apache.spark.examples ( + set EXAMPLE_CLASS=org.apache.spark.examples.%EXAMPLE_CLASS% +) + +rem Get the tail of the argument list, to skip the first one. This is surprisingly +rem complicated on Windows. +set "ARGS=" +:top +shift +if "%~1" neq "" ( + set ARGS=%ARGS% "%~1" + goto :top +) +if defined ARGS set ARGS=%ARGS:~1% -rem Figure out where java is. -set RUNNER=java -if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java +call "%FWDIR%bin\spark-submit.cmd" ^ + --master %EXAMPLE_MASTER% ^ + --class %EXAMPLE_CLASS% ^ + "%SPARK_EXAMPLES_JAR%" %ARGS% -"%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %* :exit diff --git a/bin/spark-class b/bin/spark-class index 0dcf0e156cb52..cfe363a71da31 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -24,7 +24,7 @@ esac SCALA_VERSION=2.10 -# Figure out where the Scala framework is installed +# Figure out where Spark is installed FWDIR="$(cd `dirname $0`/..; pwd)" # Export this as SPARK_HOME @@ -47,9 +47,9 @@ DEFAULT_MEM=${SPARK_MEM:-512m} SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true" -# Add java opts and memory settings for master, worker, executors, and repl. +# Add java opts and memory settings for master, worker, history server, executors, and repl. case "$1" in - # Master and Worker use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. + # Master, Worker, and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. 'org.apache.spark.deploy.master.Master') OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_MASTER_OPTS" OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} @@ -58,6 +58,10 @@ case "$1" in OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_WORKER_OPTS" OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} ;; + 'org.apache.spark.deploy.history.HistoryServer') + OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_HISTORY_OPTS" + OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} + ;; # Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. 'org.apache.spark.executor.CoarseGrainedExecutorBackend') @@ -69,11 +73,13 @@ case "$1" in OUR_JAVA_MEM=${SPARK_EXECUTOR_MEMORY:-$DEFAULT_MEM} ;; - # All drivers use SPARK_JAVA_OPTS + SPARK_DRIVER_MEMORY. The repl also uses SPARK_REPL_OPTS. - 'org.apache.spark.repl.Main') - OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_REPL_OPTS" + # Spark submit uses SPARK_SUBMIT_OPTS and SPARK_JAVA_OPTS + 'org.apache.spark.deploy.SparkSubmit') + OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_SUBMIT_OPTS \ + -Djava.library.path=$SPARK_SUBMIT_LIBRARY_PATH" OUR_JAVA_MEM=${SPARK_DRIVER_MEMORY:-$DEFAULT_MEM} ;; + *) OUR_JAVA_OPTS="$SPARK_JAVA_OPTS" OUR_JAVA_MEM=${SPARK_DRIVER_MEMORY:-$DEFAULT_MEM} @@ -93,32 +99,14 @@ else fi # Set JAVA_OPTS to be able to load native libraries and to set heap size -JAVA_OPTS="$OUR_JAVA_OPTS" -JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH" +JAVA_OPTS="-XX:MaxPermSize=128m $OUR_JAVA_OPTS" JAVA_OPTS="$JAVA_OPTS -Xms$OUR_JAVA_MEM -Xmx$OUR_JAVA_MEM" # Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e "$FWDIR/conf/java-opts" ] ; then JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`" fi export JAVA_OPTS -# Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! - -if [ ! -f "$FWDIR/RELEASE" ]; then - # Exit if the user hasn't compiled Spark - num_jars=$(ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/ | grep "spark-assembly.*hadoop.*.jar" | wc -l) - jars_list=$(ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/ | grep "spark-assembly.*hadoop.*.jar") - if [ "$num_jars" -eq "0" ]; then - echo "Failed to find Spark assembly in $FWDIR/assembly/target/scala-$SCALA_VERSION/" >&2 - echo "You need to build Spark with 'sbt/sbt assembly' before running this program." >&2 - exit 1 - fi - if [ "$num_jars" -gt "1" ]; then - echo "Found multiple Spark assembly jars in $FWDIR/assembly/target/scala-$SCALA_VERSION:" >&2 - echo "$jars_list" - echo "Please remove all but one jar." - exit 1 - fi -fi +# Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in CommandUtils.scala! TOOLS_DIR="$FWDIR"/tools SPARK_TOOLS_JAR="" @@ -133,7 +121,14 @@ if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then fi # Compute classpath using external script -CLASSPATH=`$FWDIR/bin/compute-classpath.sh` +classpath_output=$($FWDIR/bin/compute-classpath.sh) +if [[ "$?" != "0" ]]; then + echo "$classpath_output" + exit 1 +else + CLASSPATH=$classpath_output +fi + if [[ "$1" =~ org.apache.spark.tools.* ]]; then CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR" fi @@ -154,5 +149,3 @@ if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then fi exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" - - diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index f488cfdbeceb6..e420eb409e529 100755 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -17,6 +17,8 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem +setlocal enabledelayedexpansion + set SCALA_VERSION=2.10 rem Figure out where the Spark framework is installed @@ -45,14 +47,17 @@ if "x%OUR_JAVA_MEM%"=="x" set OUR_JAVA_MEM=512m set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true -rem Add java opts and memory settings for master, worker, executors, and repl. -rem Master and Worker use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. +rem Add java opts and memory settings for master, worker, history server, executors, and repl. +rem Master, Worker and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. if "%1"=="org.apache.spark.deploy.master.Master" ( set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_MASTER_OPTS% if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% ) else if "%1"=="org.apache.spark.deploy.worker.Worker" ( set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_WORKER_OPTS% if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% +) else if "%1"=="org.apache.spark.deploy.history.HistoryServer" ( + set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_HISTORY_OPTS% + if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% rem Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. ) else if "%1"=="org.apache.spark.executor.CoarseGrainedExecutorBackend" ( @@ -72,8 +77,8 @@ rem All drivers use SPARK_JAVA_OPTS + SPARK_DRIVER_MEMORY. The repl also uses SP ) rem Set JAVA_OPTS to be able to load native libraries and to set heap size -set JAVA_OPTS=%OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% -rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! +set JAVA_OPTS=-XX:MaxPermSize=128m %OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% +rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in CommandUtils.scala! rem Test whether the user has built Spark if exist "%FWDIR%RELEASE" goto skip_build_test diff --git a/bin/spark-shell b/bin/spark-shell index 535ee3ccd8269..850e9507ec38f 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -19,8 +19,6 @@ # # Shell script for starting the Spark Shell REPL -# Note that it will set MASTER to spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT} -# if those two env vars are set in spark-env.sh but MASTER is not. cygwin=false case "`uname`" in @@ -33,130 +31,13 @@ set -o posix ## Global script variables FWDIR="$(cd `dirname $0`/..; pwd)" -SPARK_REPL_OPTS="${SPARK_REPL_OPTS:-""}" -DEFAULT_MASTER="local" -MASTER=${MASTER:-""} - -info_log=0 - -#CLI Color Templates -txtund=$(tput sgr 0 1) # Underline -txtbld=$(tput bold) # Bold -bldred=${txtbld}$(tput setaf 1) # red -bldyel=${txtbld}$(tput setaf 3) # yellow -bldblu=${txtbld}$(tput setaf 4) # blue -bldwht=${txtbld}$(tput setaf 7) # white -txtrst=$(tput sgr0) # Reset -info=${bldwht}*${txtrst} # Feedback -pass=${bldblu}*${txtrst} -warn=${bldred}*${txtrst} -ques=${bldblu}?${txtrst} - -# Helper function to describe the script usage -function usage() { - cat << EOF -${txtbld}Usage${txtrst}: spark-shell [OPTIONS] - -${txtbld}OPTIONS${txtrst}: - -h --help : Print this help information. - -c --cores : The maximum number of cores to be used by the Spark Shell. - -em --executor-memory : The memory used by each executor of the Spark Shell, the number - is followed by m for megabytes or g for gigabytes, e.g. "1g". - -dm --driver-memory : The memory used by the Spark Shell, the number is followed - by m for megabytes or g for gigabytes, e.g. "1g". - -m --master : A full string that describes the Spark Master, defaults to "local" - e.g. "spark://localhost:7077". - --log-conf : Enables logging of the supplied SparkConf as INFO at start of the - Spark Context. - -e.g. - spark-shell -m spark://localhost:7077 -c 4 -dm 512m -em 2g - -EOF -} - -function out_error(){ - echo -e "${txtund}${bldred}ERROR${txtrst}: $1" - usage - exit 1 -} - -function log_info(){ - [ $info_log -eq 1 ] && echo -e "${bldyel}INFO${txtrst}: $1" -} - -function log_warn(){ - echo -e "${txtund}${bldyel}WARN${txtrst}: $1" -} - -# PATTERNS used to validate more than one optional arg. -ARG_FLAG_PATTERN="^-" -MEM_PATTERN="^[0-9]+[m|g|M|G]$" -NUM_PATTERN="^[0-9]+$" -PORT_PATTERN="^[0-9]+$" - -# Setters for optional args. -function set_cores(){ - CORE_PATTERN="^[0-9]+$" - if [[ "$1" =~ $CORE_PATTERN ]]; then - SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.cores.max=$1" - else - out_error "wrong format for $2" - fi -} - -function set_em(){ - if [[ $1 =~ $MEM_PATTERN ]]; then - SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.executor.memory=$1" - else - out_error "wrong format for $2" - fi -} - -function set_dm(){ - if [[ $1 =~ $MEM_PATTERN ]]; then - export SPARK_DRIVER_MEMORY=$1 - else - out_error "wrong format for $2" - fi -} - -function set_spark_log_conf(){ - SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.logConf=$1" -} - -function set_spark_master(){ - if ! [[ "$1" =~ $ARG_FLAG_PATTERN ]]; then - export MASTER="$1" - else - out_error "wrong format for $2" - fi -} - -function resolve_spark_master(){ - # Set MASTER from spark-env if possible - DEFAULT_SPARK_MASTER_PORT=7077 - if [ -z "$MASTER" ]; then - . $FWDIR/bin/load-spark-env.sh - if [ -n "$SPARK_MASTER_IP" ]; then - SPARK_MASTER_PORT="${SPARK_MASTER_PORT:-"$DEFAULT_SPARK_MASTER_PORT"}" - export MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}" - fi - fi - - if [ -z "$MASTER" ]; then - export MASTER="$DEFAULT_MASTER" - fi - -} +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + echo "Usage: ./bin/spark-shell [options]" + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit 0 +fi function main(){ - log_info "Base Directory set to $FWDIR" - - resolve_spark_master - log_info "Spark Master is $MASTER" - - log_info "Spark REPL options $SPARK_REPL_OPTS" if $cygwin; then # Workaround for issue involving JLine and Cygwin # (see http://sourceforge.net/p/jline/bugs/40/). @@ -164,56 +45,15 @@ function main(){ # "Backspace sends ^H" setting in "Keys" section of the Mintty options # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 - export SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-class org.apache.spark.repl.Main "$@" + export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" + $FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main stty icanon echo > /dev/null 2>&1 else - export SPARK_REPL_OPTS - $FWDIR/bin/spark-class org.apache.spark.repl.Main "$@" + export SPARK_SUBMIT_OPTS + $FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main fi } -for option in "$@" -do - case $option in - -h | --help ) - usage - exit 1 - ;; - -c | --cores) - shift - _1=$1 - shift - set_cores $_1 "-c/--cores" - ;; - -em | --executor-memory) - shift - _1=$1 - shift - set_em $_1 "-em/--executor-memory" - ;; - -dm | --driver-memory) - shift - _1=$1 - shift - set_dm $_1 "-dm/--driver-memory" - ;; - -m | --master) - shift - _1=$1 - shift - set_spark_master $_1 "-m/--master" - ;; - --log-conf) - shift - set_spark_log_conf "true" - info_log=1 - ;; - ?) - ;; - esac -done - # Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in # binary distribution of Spark where Scala is not installed exit_status=127 @@ -242,7 +82,7 @@ if [[ ! $? ]]; then saved_stty="" fi -main +main "$@" # record the exit status lest it be overwritten: # then reenable echo and propagate the code. diff --git a/bin/spark-shell.cmd b/bin/spark-shell.cmd index 99799128eb734..4b9708a8c03f3 100755 --- a/bin/spark-shell.cmd +++ b/bin/spark-shell.cmd @@ -17,7 +17,6 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem -rem Find the path of sbin -set BIN=%~dp0..\bin\ +set SPARK_HOME=%~dp0.. -cmd /V /E /C %BIN%spark-class2.cmd org.apache.spark.repl.Main %* +cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-shell %* --class org.apache.spark.repl.Main diff --git a/bin/spark-submit b/bin/spark-submit index d92d55a032bd5..9e7cecedd0325 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -18,21 +18,28 @@ # export SPARK_HOME="$(cd `dirname $0`/..; pwd)" -ORIG_ARGS=$@ +ORIG_ARGS=("$@") while (($#)); do - if [ $1 = "--deploy-mode" ]; then + if [ "$1" = "--deploy-mode" ]; then DEPLOY_MODE=$2 - elif [ $1 = "--driver-memory" ]; then + elif [ "$1" = "--driver-memory" ]; then DRIVER_MEMORY=$2 + elif [ "$1" = "--driver-library-path" ]; then + export SPARK_SUBMIT_LIBRARY_PATH=$2 + elif [ "$1" = "--driver-class-path" ]; then + export SPARK_SUBMIT_CLASSPATH=$2 + elif [ "$1" = "--driver-java-options" ]; then + export SPARK_SUBMIT_OPTS=$2 fi - shift done -if [ ! -z $DRIVER_MEMORY ] && [ ! -z $DEPLOY_MODE ] && [ $DEPLOY_MODE = "client" ]; then - export SPARK_MEM=$DRIVER_MEMORY +DEPLOY_MODE=${DEPLOY_MODE:-"client"} + +if [ -n "$DRIVER_MEMORY" ] && [ $DEPLOY_MODE == "client" ]; then + export SPARK_DRIVER_MEMORY=$DRIVER_MEMORY fi -$SPARK_HOME/bin/spark-class org.apache.spark.deploy.SparkSubmit $ORIG_ARGS +exec $SPARK_HOME/bin/spark-class org.apache.spark.deploy.SparkSubmit "${ORIG_ARGS[@]}" diff --git a/bin/spark-submit.cmd b/bin/spark-submit.cmd new file mode 100644 index 0000000000000..6eb702ed8c561 --- /dev/null +++ b/bin/spark-submit.cmd @@ -0,0 +1,56 @@ +@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 + +set SPARK_HOME=%~dp0.. +set ORIG_ARGS=%* + +rem Clear the values of all variables used +set DEPLOY_MODE= +set DRIVER_MEMORY= +set SPARK_SUBMIT_LIBRARY_PATH= +set SPARK_SUBMIT_CLASSPATH= +set SPARK_SUBMIT_OPTS= +set SPARK_DRIVER_MEMORY= + +:loop +if [%1] == [] goto continue + if [%1] == [--deploy-mode] ( + set DEPLOY_MODE=%2 + ) else if [%1] == [--driver-memory] ( + set DRIVER_MEMORY=%2 + ) else if [%1] == [--driver-library-path] ( + set SPARK_SUBMIT_LIBRARY_PATH=%2 + ) else if [%1] == [--driver-class-path] ( + set SPARK_SUBMIT_CLASSPATH=%2 + ) else if [%1] == [--driver-java-options] ( + set SPARK_SUBMIT_OPTS=%2 + ) + shift +goto loop +:continue + +if [%DEPLOY_MODE%] == [] ( + set DEPLOY_MODE=client +) + +if not [%DRIVER_MEMORY%] == [] if [%DEPLOY_MODE%] == [client] ( + set SPARK_DRIVER_MEMORY=%DRIVER_MEMORY% +) + +cmd /V /E /C %SPARK_HOME%\bin\spark-class.cmd org.apache.spark.deploy.SparkSubmit %ORIG_ARGS% diff --git a/conf/log4j.properties.template b/conf/log4j.properties.template index f7f853559468a..89eec7d4b7f61 100644 --- a/conf/log4j.properties.template +++ b/conf/log4j.properties.template @@ -7,5 +7,6 @@ log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: # 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.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO diff --git a/conf/spark-defaults.conf.template b/conf/spark-defaults.conf.template new file mode 100644 index 0000000000000..2779342769c14 --- /dev/null +++ b/conf/spark-defaults.conf.template @@ -0,0 +1,8 @@ +# Default system properties included when running spark-submit. +# This is useful for setting default environmental settings. + +# Example: +# spark.master spark://master:7077 +# spark.eventLog.enabled true +# spark.eventLog.dir hdfs://namenode:8021/directory +# spark.serializer org.apache.spark.serializer.KryoSerializer diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 6432a566089be..f8ffbf64278fb 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -1,22 +1,43 @@ #!/usr/bin/env bash -# This file contains environment variables required to run Spark. Copy it as -# spark-env.sh and edit that to configure Spark for your site. -# -# The following variables can be set in this file: +# 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 +# ./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 +# - SPARK_PUBLIC_DNS, to set the public dns name of the driver program +# - SPARK_CLASSPATH, default classpath entries to append + +# Options read by executors and drivers running inside the cluster # - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node +# - SPARK_PUBLIC_DNS, to set the public DNS name of the driver program +# - SPARK_CLASSPATH, default classpath entries to append +# - SPARK_LOCAL_DIRS, storage directories to use on this node for shuffle and RDD data # - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos -# - SPARK_JAVA_OPTS, to set node-specific JVM options for Spark. Note that -# we recommend setting app-wide options in the application's driver program. -# Examples of node-specific options : -Dspark.local.dir, GC options -# Examples of app-wide options : -Dspark.serializer -# -# If using the standalone deploy mode, you can also set variables for it here: + +# Options read in YARN client mode +# - HADOOP_CONF_DIR, to point Spark towards Hadoop configuration files +# - SPARK_EXECUTOR_INSTANCES, Number of workers to start (Default: 2) +# - SPARK_EXECUTOR_CORES, Number of cores for the workers (Default: 1). +# - SPARK_EXECUTOR_MEMORY, Memory per Worker (e.g. 1000M, 2G) (Default: 1G) +# - SPARK_DRIVER_MEMORY, Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb) +# - SPARK_YARN_APP_NAME, The name of your application (Default: Spark) +# - SPARK_YARN_QUEUE, The hadoop queue to use for allocation requests (Default: ‘default’) +# - SPARK_YARN_DIST_FILES, Comma separated list of files to be distributed with the job. +# - SPARK_YARN_DIST_ARCHIVES, Comma separated list of archives to be distributed with the job. + +# Options for the daemons used in the standalone deploy mode: # - SPARK_MASTER_IP, to bind the master to a different IP address or hostname -# - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports +# - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports for the master +# - SPARK_MASTER_OPTS, to set config properties only for the master (e.g. "-Dx=y") # - SPARK_WORKER_CORES, to set the number of cores to use on this machine -# - SPARK_WORKER_MEMORY, to set how much memory to use (e.g. 1000m, 2g) -# - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT +# - SPARK_WORKER_MEMORY, to set how much total memory workers have to give executors (e.g. 1000m, 2g) +# - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT, to use non-default ports for the worker # - SPARK_WORKER_INSTANCES, to set the number of worker processes per node # - SPARK_WORKER_DIR, to set the working directory of worker processes -# - SPARK_PUBLIC_DNS, to set the public dns name of the master +# - 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_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 66f9fc4961b03..8c23842730e37 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml @@ -30,19 +30,6 @@ jar Spark Project Core http://spark.apache.org/ - - - - yarn-alpha - - - org.apache.avro - avro - - - - - org.apache.hadoop @@ -51,12 +38,6 @@ net.java.dev.jets3t jets3t - - - commons-logging - commons-logging - - org.apache.curator @@ -82,6 +63,16 @@ com.google.guava guava + + org.apache.commons + commons-lang3 + + + org.apache.commons + commons-math3 + 3.3 + test + com.google.code.findbugs jsr305 @@ -117,12 +108,10 @@ com.twitter chill_${scala.binary.version} - 0.3.1 com.twitter chill-java - 0.3.1 commons-net @@ -149,19 +138,6 @@ org.json4s json4s-jackson_${scala.binary.version} 3.2.6 - - - - org.scala-lang - scalap - - - - - it.unimi.dsi - fastutil colt @@ -170,6 +146,7 @@ org.apache.mesos mesos + ${mesos.classifier} io.netty @@ -264,7 +241,12 @@ org.easymock - easymock + easymockclassextension + test + + + asm + asm test @@ -272,40 +254,21 @@ junit-interface test + + org.spark-project + pyrolite + 2.0.1 + + + net.sf.py4j + py4j + 0.8.1 + target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - org.apache.maven.plugins - maven-antrun-plugin - - - test - - run - - - true - - - - - - - - - - - - - - - - - - org.scalatest scalatest-maven-plugin @@ -317,6 +280,48 @@ + + + org.codehaus.mojo + exec-maven-plugin + 1.2.1 + + + generate-resources + + exec + + + + + unzip + ../python + + -o + lib/py4j*.zip + -d + build + + + + + + + src/main/resources + + + ../python + + pyspark/*.py + + + + ../python/build + + py4j/*.py + + + diff --git a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java index fa75842047c6a..23f5fdd43631b 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java +++ b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java @@ -24,4 +24,4 @@ */ public interface FlatMapFunction extends Serializable { public Iterable call(T t) throws Exception; -} \ No newline at end of file +} diff --git a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java index d1fdec072443d..c48e92f535ff5 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java +++ b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java @@ -24,4 +24,4 @@ */ public interface FlatMapFunction2 extends Serializable { public Iterable call(T1 t1, T2 t2) throws Exception; -} \ No newline at end of file +} diff --git a/core/src/main/java/org/apache/spark/api/java/function/package-info.java b/core/src/main/java/org/apache/spark/api/java/function/package-info.java new file mode 100644 index 0000000000000..463a42f23342c --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 of interfaces to represent functions in Spark's Java API. Users create implementations of + * these interfaces to pass functions to various Java API methods for Spark. Please visit Spark's + * Java programming guide for more details. + */ +package org.apache.spark.api.java.function; \ No newline at end of file diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala b/core/src/main/java/org/apache/spark/api/java/function/package.scala similarity index 73% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala rename to core/src/main/java/org/apache/spark/api/java/function/package.scala index 416996fcbe760..7f91de653a64a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala +++ b/core/src/main/java/org/apache/spark/api/java/function/package.scala @@ -15,13 +15,11 @@ * limitations under the License. */ -package org.apache.spark.mllib.linalg +package org.apache.spark.api.java /** - * Class that represents an entry in a sparse matrix of doubles. - * - * @param i row index (0 indexing used) - * @param j column index (0 indexing used) - * @param mval value of entry in matrix + * Set of interfaces to represent functions in Spark's Java API. Users create implementations of + * these interfaces to pass functions to various Java API methods for Spark. Please visit Spark's + * Java programming guide for more details. */ -case class MatrixEntry(val i: Int, val j: Int, val mval: Double) +package object function \ No newline at end of file diff --git a/core/src/main/java/org/apache/spark/package-info.java b/core/src/main/java/org/apache/spark/package-info.java new file mode 100644 index 0000000000000..4426c7afcebdd --- /dev/null +++ b/core/src/main/java/org/apache/spark/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Core Spark classes in Scala. A few classes here, such as {@link org.apache.spark.Accumulator} + * and {@link org.apache.spark.storage.StorageLevel}, are also used in Java, but the + * {@link org.apache.spark.api.java} package contains the main Java API. + */ +package org.apache.spark; 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 f7f853559468a..89eec7d4b7f61 100644 --- a/core/src/main/resources/org/apache/spark/log4j-defaults.properties +++ b/core/src/main/resources/org/apache/spark/log4j-defaults.properties @@ -7,5 +7,6 @@ log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: # 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.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/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index fe54c34ffb1da..a8bc141208a94 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 @@ -78,3 +78,33 @@ table.sortable thead { background-repeat: repeat-x; filter: progid:dximagetransform.microsoft.gradient(startColorstr='#FFA4EDFF', endColorstr='#FF94DDFF', GradientType=0); } + +span.kill-link { + margin-right: 2px; + color: gray; +} + +span.kill-link a { + color: gray; +} + +span.expand-details { + font-size: 10pt; + cursor: pointer; + color: grey; + float: right; +} + +.stage-details { + max-height: 100px; + overflow-y: auto; + margin: 0; + transition: max-height 0.5s ease-out, padding 0.5s ease-out; +} + +.stage-details.collapsed { + max-height: 0; + padding-top: 0; + padding-bottom: 0; + border: none; +} diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index d5f3e3f6ec496..cdfd338081fa2 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -21,6 +21,7 @@ import java.io.{ObjectInputStream, Serializable} import scala.collection.generic.Growable import scala.collection.mutable.Map +import scala.reflect.ClassTag import org.apache.spark.serializer.JavaSerializer @@ -104,8 +105,11 @@ class Accumulable[R, T] ( * Set the accumulator's value; only allowed on master. */ def value_= (newValue: R) { - if (!deserialized) value_ = newValue - else throw new UnsupportedOperationException("Can't assign accumulator value in task") + if (!deserialized) { + value_ = newValue + } else { + throw new UnsupportedOperationException("Can't assign accumulator value in task") + } } /** @@ -161,9 +165,9 @@ trait AccumulableParam[R, T] extends Serializable { def zero(initialValue: R): R } -private[spark] -class GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializable, T] - extends AccumulableParam[R,T] { +private[spark] class +GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializable: ClassTag, T] + extends AccumulableParam[R, T] { def addAccumulator(growable: R, elem: T): R = { growable += elem diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index ceead59b79ed6..59fdf659c9e11 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -17,15 +17,18 @@ package org.apache.spark +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** + * :: DeveloperApi :: * A set of functions used to aggregate data. * * @param createCombiner function to create the initial value of the aggregation. * @param mergeValue function to merge a new value into the aggregation result. * @param mergeCombiners function to merge outputs from multiple mergeValue function. */ +@DeveloperApi case class Aggregator[K, V, C] ( createCombiner: V => C, mergeValue: (C, V) => C, diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index c7893f288b4b5..3f667a4a0f9c5 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -20,94 +20,54 @@ package org.apache.spark import scala.collection.mutable.{ArrayBuffer, HashSet} import org.apache.spark.rdd.RDD -import org.apache.spark.storage.{BlockId, BlockManager, BlockStatus, RDDBlockId, StorageLevel} +import org.apache.spark.storage._ /** - * Spark class responsible for passing RDDs split contents to the BlockManager and making + * Spark class responsible for passing RDDs partition contents to the BlockManager and making * sure a node doesn't load two copies of an RDD at once. */ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { - /** Keys of RDD splits that are being computed/loaded. */ + /** Keys of RDD partitions that are being computed/loaded. */ private val loading = new HashSet[RDDBlockId]() - /** Gets or computes an RDD split. Used by RDD.iterator() when an RDD is cached. */ - def getOrCompute[T](rdd: RDD[T], split: Partition, context: TaskContext, + /** Gets or computes an RDD partition. Used by RDD.iterator() when an RDD is cached. */ + def getOrCompute[T]( + rdd: RDD[T], + partition: Partition, + context: TaskContext, storageLevel: StorageLevel): Iterator[T] = { - val key = RDDBlockId(rdd.id, split.index) - logDebug("Looking for partition " + key) + + val key = RDDBlockId(rdd.id, partition.index) + logDebug(s"Looking for partition $key") blockManager.get(key) match { case Some(values) => // Partition is already materialized, so just return its values new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]]) case None => - // Mark the split as loading (unless someone else marks it first) - loading.synchronized { - if (loading.contains(key)) { - logInfo("Another thread is loading %s, waiting for it to finish...".format(key)) - while (loading.contains(key)) { - try {loading.wait()} catch {case _ : Throwable =>} - } - logInfo("Finished waiting for %s".format(key)) - /* See whether someone else has successfully loaded it. The main way this would fail - * is for the RDD-level cache eviction policy if someone else has loaded the same RDD - * partition but we didn't want to make space for it. However, that case is unlikely - * because it's unlikely that two threads would work on the same RDD partition. One - * downside of the current code is that threads wait serially if this does happen. */ - blockManager.get(key) match { - case Some(values) => - return new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]]) - case None => - logInfo("Whoever was loading %s failed; we'll try it ourselves".format(key)) - loading.add(key) - } - } else { - loading.add(key) - } + // Acquire a lock for loading this partition + // If another thread already holds the lock, wait for it to finish return its results + val storedValues = acquireLockForPartition[T](key) + if (storedValues.isDefined) { + return new InterruptibleIterator[T](context, storedValues.get) } + + // Otherwise, we have to load the partition ourselves try { - // If we got here, we have to load the split - logInfo("Partition %s not found, computing it".format(key)) - val computedValues = rdd.computeOrReadCheckpoint(split, context) - - // Persist the result, so long as the task is not running locally - if (context.runningLocally) { return computedValues } - - // Keep track of blocks with updated statuses - var updatedBlocks = Seq[(BlockId, BlockStatus)]() - val returnValue: Iterator[T] = { - if (storageLevel.useDisk && !storageLevel.useMemory) { - /* In the case that this RDD is to be persisted using DISK_ONLY - * the iterator will be passed directly to the blockManager (rather then - * caching it to an ArrayBuffer first), then the resulting block data iterator - * will be passed back to the user. If the iterator generates a lot of data, - * this means that it doesn't all have to be held in memory at one time. - * This could also apply to MEMORY_ONLY_SER storage, but we need to make sure - * blocks aren't dropped by the block store before enabling that. */ - updatedBlocks = blockManager.put(key, computedValues, storageLevel, tellMaster = true) - blockManager.get(key) match { - case Some(values) => - new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]]) - case None => - logInfo("Failure to store %s".format(key)) - throw new Exception("Block manager failed to return persisted valued") - } - } else { - // In this case the RDD is cached to an array buffer. This will save the results - // if we're dealing with a 'one-time' iterator - val elements = new ArrayBuffer[Any] - elements ++= computedValues - updatedBlocks = blockManager.put(key, elements, storageLevel, tellMaster = true) - elements.iterator.asInstanceOf[Iterator[T]] - } - } + logInfo(s"Partition $key not found, computing it") + val computedValues = rdd.computeOrReadCheckpoint(partition, context) - // Update task metrics to include any blocks whose storage status is updated - val metrics = context.taskMetrics - metrics.updatedBlocks = Some(updatedBlocks) + // If the task is running locally, do not persist the result + if (context.runningLocally) { + return computedValues + } - returnValue + // Otherwise, cache the values and keep track of any updates in block statuses + val updatedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + val cachedValues = putInBlockManager(key, computedValues, storageLevel, updatedBlocks) + context.taskMetrics.updatedBlocks = Some(updatedBlocks) + new InterruptibleIterator(context, cachedValues) } finally { loading.synchronized { @@ -117,4 +77,76 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { } } } + + /** + * Acquire a loading lock for the partition identified by the given block ID. + * + * If the lock is free, just acquire it and return None. Otherwise, another thread is already + * loading the partition, so we wait for it to finish and return the values loaded by the thread. + */ + private def acquireLockForPartition[T](id: RDDBlockId): Option[Iterator[T]] = { + loading.synchronized { + if (!loading.contains(id)) { + // If the partition is free, acquire its lock to compute its value + loading.add(id) + None + } else { + // Otherwise, wait for another thread to finish and return its result + logInfo(s"Another thread is loading $id, waiting for it to finish...") + while (loading.contains(id)) { + try { + loading.wait() + } catch { + case e: Exception => + logWarning(s"Exception while waiting for another thread to load $id", e) + } + } + logInfo(s"Finished waiting for $id") + val values = blockManager.get(id) + if (!values.isDefined) { + /* The block is not guaranteed to exist even after the other thread has finished. + * For instance, the block could be evicted after it was put, but before our get. + * In this case, we still need to load the partition ourselves. */ + logInfo(s"Whoever was loading $id failed; we'll try it ourselves") + loading.add(id) + } + values.map(_.asInstanceOf[Iterator[T]]) + } + } + } + + /** + * Cache the values of a partition, keeping track of any updates in the storage statuses + * of other blocks along the way. + */ + private def putInBlockManager[T]( + key: BlockId, + values: Iterator[T], + storageLevel: StorageLevel, + updatedBlocks: ArrayBuffer[(BlockId, BlockStatus)]): Iterator[T] = { + + if (!storageLevel.useMemory) { + /* This RDD is not to be cached in memory, so we can just pass the computed values + * as an iterator directly to the BlockManager, rather than first fully unrolling + * it in memory. The latter option potentially uses much more memory and risks OOM + * exceptions that can be avoided. */ + updatedBlocks ++= blockManager.put(key, values, storageLevel, tellMaster = true) + blockManager.get(key) match { + case Some(v) => v.asInstanceOf[Iterator[T]] + case None => + logInfo(s"Failure to store $key") + throw new BlockException(key, s"Block manager failed to return cached value for $key!") + } + } else { + /* This RDD is to be cached in memory. In this case we cannot pass the computed values + * to the BlockManager as an iterator and expect to read it back later. This is because + * we may end up dropping a partition from memory store before getting it back, e.g. + * when the entirety of the RDD does not fit in memory. */ + val elements = new ArrayBuffer[Any] + elements ++= values + updatedBlocks ++= blockManager.put(key, elements, storageLevel, tellMaster = true) + elements.iterator.asInstanceOf[Iterator[T]] + } + } + } diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala new file mode 100644 index 0000000000000..bf3c3a6ceb5ef --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import java.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.util.Utils + +/** + * Classes that represent cleaning tasks. + */ +private sealed trait CleanupTask +private case class CleanRDD(rddId: Int) extends CleanupTask +private case class CleanShuffle(shuffleId: Int) extends CleanupTask +private case class CleanBroadcast(broadcastId: Long) extends CleanupTask + +/** + * A WeakReference associated with a CleanupTask. + * + * When the referent object becomes only weakly reachable, the corresponding + * CleanupTaskWeakReference is automatically added to the given reference queue. + */ +private class CleanupTaskWeakReference( + val task: CleanupTask, + referent: AnyRef, + referenceQueue: ReferenceQueue[AnyRef]) + extends WeakReference(referent, referenceQueue) + +/** + * An asynchronous cleaner for RDD, shuffle, and broadcast state. + * + * This maintains a weak reference for each RDD, ShuffleDependency, and Broadcast of interest, + * to be processed when the associated object goes out of scope of the application. Actual + * cleanup is performed in a separate daemon thread. + */ +private[spark] class ContextCleaner(sc: SparkContext) extends Logging { + + private val referenceBuffer = new ArrayBuffer[CleanupTaskWeakReference] + with SynchronizedBuffer[CleanupTaskWeakReference] + + private val referenceQueue = new ReferenceQueue[AnyRef] + + private val listeners = new ArrayBuffer[CleanerListener] + with SynchronizedBuffer[CleanerListener] + + private val cleaningThread = new Thread() { override def run() { keepCleaning() }} + + /** + * Whether the cleaning thread will block on cleanup tasks. + * This is set to true only for tests. + */ + private val blockOnCleanupTasks = sc.conf.getBoolean( + "spark.cleaner.referenceTracking.blocking", false) + + @volatile private var stopped = false + + /** Attach a listener object to get information of when objects are cleaned. */ + def attachListener(listener: CleanerListener) { + listeners += listener + } + + /** Start the cleaner. */ + def start() { + cleaningThread.setDaemon(true) + cleaningThread.setName("Spark Context Cleaner") + cleaningThread.start() + } + + /** Stop the cleaner. */ + def stop() { + stopped = true + } + + /** Register a RDD for cleanup when it is garbage collected. */ + def registerRDDForCleanup(rdd: RDD[_]) { + registerForCleanup(rdd, CleanRDD(rdd.id)) + } + + /** Register a ShuffleDependency for cleanup when it is garbage collected. */ + def registerShuffleForCleanup(shuffleDependency: ShuffleDependency[_, _, _]) { + registerForCleanup(shuffleDependency, CleanShuffle(shuffleDependency.shuffleId)) + } + + /** Register a Broadcast for cleanup when it is garbage collected. */ + def registerBroadcastForCleanup[T](broadcast: Broadcast[T]) { + registerForCleanup(broadcast, CleanBroadcast(broadcast.id)) + } + + /** Register an object for cleanup. */ + private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask) { + referenceBuffer += new CleanupTaskWeakReference(task, objectForCleanup, referenceQueue) + } + + /** Keep cleaning RDD, shuffle, and broadcast state. */ + private def keepCleaning(): Unit = Utils.logUncaughtExceptions { + while (!stopped) { + try { + val reference = Option(referenceQueue.remove(ContextCleaner.REF_QUEUE_POLL_TIMEOUT)) + .map(_.asInstanceOf[CleanupTaskWeakReference]) + reference.map(_.task).foreach { task => + logDebug("Got cleaning task " + task) + referenceBuffer -= reference.get + task match { + case CleanRDD(rddId) => + doCleanupRDD(rddId, blocking = blockOnCleanupTasks) + case CleanShuffle(shuffleId) => + doCleanupShuffle(shuffleId, blocking = blockOnCleanupTasks) + case CleanBroadcast(broadcastId) => + doCleanupBroadcast(broadcastId, blocking = blockOnCleanupTasks) + } + } + } catch { + case e: Exception => logError("Error in cleaning thread", e) + } + } + } + + /** Perform RDD cleanup. */ + def doCleanupRDD(rddId: Int, blocking: Boolean) { + try { + logDebug("Cleaning RDD " + rddId) + sc.unpersistRDD(rddId, blocking) + listeners.foreach(_.rddCleaned(rddId)) + logInfo("Cleaned RDD " + rddId) + } catch { + case e: Exception => logError("Error cleaning RDD " + rddId, e) + } + } + + /** Perform shuffle cleanup, asynchronously. */ + def doCleanupShuffle(shuffleId: Int, blocking: Boolean) { + try { + logDebug("Cleaning shuffle " + shuffleId) + mapOutputTrackerMaster.unregisterShuffle(shuffleId) + blockManagerMaster.removeShuffle(shuffleId, blocking) + listeners.foreach(_.shuffleCleaned(shuffleId)) + logInfo("Cleaned shuffle " + shuffleId) + } catch { + case e: Exception => logError("Error cleaning shuffle " + shuffleId, e) + } + } + + /** Perform broadcast cleanup. */ + def doCleanupBroadcast(broadcastId: Long, blocking: Boolean) { + try { + logDebug("Cleaning broadcast " + broadcastId) + broadcastManager.unbroadcast(broadcastId, true, blocking) + listeners.foreach(_.broadcastCleaned(broadcastId)) + logInfo("Cleaned broadcast " + broadcastId) + } catch { + case e: Exception => logError("Error cleaning broadcast " + broadcastId, e) + } + } + + private def blockManagerMaster = sc.env.blockManager.master + private def broadcastManager = sc.env.broadcastManager + private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] + + // Used for testing. These methods explicitly blocks until cleanup is completed + // to ensure that more reliable testing. +} + +private object ContextCleaner { + private val REF_QUEUE_POLL_TIMEOUT = 100 +} + +/** + * Listener class used for testing when any item has been cleaned by the Cleaner class. + */ +private[spark] trait CleanerListener { + def rddCleaned(rddId: Int) + def shuffleCleaned(shuffleId: Int) + def broadcastCleaned(broadcastId: Long) +} diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 3132dcf745e19..c8c194a111aac 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -17,19 +17,25 @@ package org.apache.spark +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.ShuffleHandle /** + * :: DeveloperApi :: * Base class for dependencies. */ +@DeveloperApi abstract class Dependency[T](val rdd: RDD[T]) extends Serializable /** + * :: DeveloperApi :: * Base class for dependencies where each partition of the parent RDD is used by at most one * partition of the child RDD. Narrow dependencies allow for pipelined execution. */ +@DeveloperApi abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** * Get the parent partitions for a child partition. @@ -41,38 +47,51 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** + * :: DeveloperApi :: * Represents a dependency on the output of a shuffle stage. * @param rdd the parent RDD * @param partitioner partitioner used to partition the shuffle output - * @param serializer [[org.apache.spark.serializer.Serializer Serializer]] to use. If set to null, + * @param serializer [[org.apache.spark.serializer.Serializer Serializer]] to use. If set to None, * the default serializer, as specified by `spark.serializer` config option, will * be used. */ -class ShuffleDependency[K, V]( +@DeveloperApi +class ShuffleDependency[K, V, C]( @transient rdd: RDD[_ <: Product2[K, V]], val partitioner: Partitioner, - val serializer: Serializer = null) + val serializer: Option[Serializer] = None, + val keyOrdering: Option[Ordering[K]] = None, + val aggregator: Option[Aggregator[K, V, C]] = None) extends Dependency(rdd.asInstanceOf[RDD[Product2[K, V]]]) { val shuffleId: Int = rdd.context.newShuffleId() + + val shuffleHandle: ShuffleHandle = rdd.context.env.shuffleManager.registerShuffle( + shuffleId, rdd.partitions.size, this) + + rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this)) } /** + * :: DeveloperApi :: * Represents a one-to-one dependency between partitions of the parent and child RDDs. */ +@DeveloperApi class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { override def getParents(partitionId: Int) = List(partitionId) } /** + * :: DeveloperApi :: * Represents a one-to-one dependency between ranges of partitions in the parent and child RDDs. * @param rdd the parent RDD * @param inStart the start of the range in the parent RDD * @param outStart the start of the range in the child RDD * @param length the length of the range */ +@DeveloperApi class RangeDependency[T](rdd: RDD[T], inStart: Int, outStart: Int, length: Int) extends NarrowDependency[T](rdd) { diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index f2decd14ef6d9..1e4dec86a0530 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -21,13 +21,16 @@ import scala.concurrent._ import scala.concurrent.duration.Duration import scala.util.Try +import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{JobFailed, JobSucceeded, JobWaiter} /** + * :: Experimental :: * A future for the result of an action to support cancellation. This is an extension of the * Scala Future interface to support cancellation. */ +@Experimental trait FutureAction[T] extends Future[T] { // Note that we redefine methods of the Future trait here explicitly so we can specify a different // documentation (with reference to the word "action"). @@ -84,9 +87,11 @@ trait FutureAction[T] extends Future[T] { /** + * :: Experimental :: * A [[FutureAction]] holding the result of an action that triggers a single job. Examples include * count, collect, reduce. */ +@Experimental class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: => T) extends FutureAction[T] { @@ -141,17 +146,19 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: private def awaitResult(): Try[T] = { jobWaiter.awaitResult() match { case JobSucceeded => scala.util.Success(resultFunc) - case JobFailed(e: Exception, _) => scala.util.Failure(e) + case JobFailed(e: Exception) => scala.util.Failure(e) } } } /** + * :: Experimental :: * A [[FutureAction]] for actions that could trigger multiple Spark jobs. Examples include take, * takeSample. Cancellation works by setting the cancelled flag to true and interrupting the * action thread if it is being blocked by a job. */ +@Experimental class ComplexFutureAction[T] extends FutureAction[T] { // Pointer to the thread that is executing the action. It is set when the action is run. diff --git a/core/src/main/scala/org/apache/spark/HttpFileServer.scala b/core/src/main/scala/org/apache/spark/HttpFileServer.scala index 3d7692ea8a49e..0e3750fdde415 100644 --- a/core/src/main/scala/org/apache/spark/HttpFileServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpFileServer.scala @@ -24,13 +24,13 @@ import com.google.common.io.Files import org.apache.spark.util.Utils private[spark] class HttpFileServer(securityManager: SecurityManager) extends Logging { - + var baseDir : File = null var fileDir : File = null var jarDir : File = null var httpServer : HttpServer = null var serverUri : String = null - + def initialize() { baseDir = Utils.createTempDir() fileDir = new File(baseDir, "files") @@ -43,24 +43,31 @@ private[spark] class HttpFileServer(securityManager: SecurityManager) extends Lo serverUri = httpServer.uri logDebug("HTTP file server started at: " + serverUri) } - + def stop() { httpServer.stop() } - + def addFile(file: File) : String = { addFileToDir(file, fileDir) serverUri + "/files/" + file.getName } - + def addJar(file: File) : String = { addFileToDir(file, jarDir) serverUri + "/jars/" + file.getName } - + def addFileToDir(file: File, dir: File) : String = { + // Check whether the file is a directory. If it is, throw a more meaningful exception. + // If we don't catch this, Guava throws a very confusing error message: + // java.io.FileNotFoundException: [file] (No such file or directory) + // even though the directory ([file]) exists. + if (file.isDirectory) { + throw new IllegalArgumentException(s"$file cannot be a directory.") + } Files.copy(file, new File(dir, file.getName)) dir + "/" + file.getName } - + } diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index cb5df25fa48df..7e9b517f901a2 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -83,19 +83,19 @@ private[spark] class HttpServer(resourceBase: File, securityManager: SecurityMan } } - /** + /** * Setup Jetty to the HashLoginService using a single user with our * shared secret. Configure it to use DIGEST-MD5 authentication so that the password * isn't passed in plaintext. */ private def setupSecurityHandler(securityMgr: SecurityManager): ConstraintSecurityHandler = { val constraint = new Constraint() - // use DIGEST-MD5 as the authentication mechanism + // use DIGEST-MD5 as the authentication mechanism constraint.setName(Constraint.__DIGEST_AUTH) constraint.setRoles(Array("user")) constraint.setAuthenticate(true) constraint.setDataConstraint(Constraint.DC_NONE) - + val cm = new ConstraintMapping() cm.setConstraint(constraint) cm.setPathSpec("/*") diff --git a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala index 9b1601d5b95fa..f40baa8e43592 100644 --- a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala +++ b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala @@ -17,14 +17,28 @@ package org.apache.spark +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * An iterator that wraps around an existing iterator to provide task killing functionality. * It works by checking the interrupted flag in [[TaskContext]]. */ +@DeveloperApi class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator[T]) extends Iterator[T] { - def hasNext: Boolean = !context.interrupted && delegate.hasNext + def hasNext: Boolean = { + // TODO(aarondav/rxin): Check Thread.interrupted instead of context.interrupted if interrupt + // is allowed. The assumption is that Thread.interrupted does not have a memory fence in read + // (just a volatile field in C), while context.interrupted is a volatile in the JVM, which + // introduces an expensive read fence. + if (context.interrupted) { + throw new TaskKilledException + } else { + delegate.hasNext + } + } def next(): T = delegate.next() } diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 7423082e34f47..50d8e93e1f0d7 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -21,11 +21,19 @@ import org.apache.log4j.{LogManager, PropertyConfigurator} import org.slf4j.{Logger, LoggerFactory} import org.slf4j.impl.StaticLoggerBinder +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.Utils + /** + * :: DeveloperApi :: * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows * logging messages at different levels using methods that only evaluate parameters lazily if the * log level is enabled. + * + * NOTE: DO NOT USE this class outside of Spark. It is intended as an internal utility. + * This will likely be changed or removed in future releases. */ +@DeveloperApi trait Logging { // Make the log field transient so that objects with Logging can // be serialized and used on another machine @@ -53,7 +61,7 @@ trait Logging { protected def logDebug(msg: => String) { if (log.isDebugEnabled) log.debug(msg) } - + protected def logTrace(msg: => String) { if (log.isTraceEnabled) log.trace(msg) } @@ -108,12 +116,11 @@ trait Logging { val log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements if (!log4jInitialized && usingLog4j) { val defaultLogProps = "org/apache/spark/log4j-defaults.properties" - val classLoader = this.getClass.getClassLoader - Option(classLoader.getResource(defaultLogProps)) match { - case Some(url) => + Option(Utils.getSparkClassLoader.getResource(defaultLogProps)) match { + case Some(url) => PropertyConfigurator.configure(url) log.info(s"Using Spark's default log4j profile: $defaultLogProps") - case None => + case None => System.err.println(s"Spark was unable to load $defaultLogProps") } } @@ -128,4 +135,16 @@ trait Logging { private object Logging { @volatile private var initialized = false val initLock = new Object() + try { + // We use reflection here to handle the case where users remove the + // slf4j-to-jul bridge order to route their logs to JUL. + val bridgeClass = Class.forName("org.slf4j.bridge.SLF4JBridgeHandler") + bridgeClass.getMethod("removeHandlersForRootLogger").invoke(null) + val installed = bridgeClass.getMethod("isInstalled").invoke(null).asInstanceOf[Boolean] + if (!installed) { + bridgeClass.getMethod("install").invoke(null) + } + } catch { + case e: ClassNotFoundException => // can't log anything yet so just fail silently + } } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 80cbf951cb70e..ee82d9fa7874b 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -20,21 +20,21 @@ package org.apache.spark import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} -import scala.collection.mutable.HashSet +import scala.collection.mutable.{HashSet, HashMap, Map} import scala.concurrent.Await import akka.actor._ import akka.pattern.ask - import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{AkkaUtils, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} +import org.apache.spark.util._ private[spark] sealed trait MapOutputTrackerMessage 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 Logging { val maxAkkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) @@ -65,26 +65,41 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster } } -private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { - +/** + * Class that keeps track of the location of the map output of + * a stage. This is abstract because different versions of MapOutputTracker + * (driver and worker) use different HashMap to store its metadata. + */ +private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging { private val timeout = AkkaUtils.askTimeout(conf) - // Set to the MapOutputTrackerActor living on the driver + /** Set to the MapOutputTrackerActor living on the driver. */ var trackerActor: ActorRef = _ - protected val mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]] + /** + * This HashMap has different behavior for the master and the workers. + * + * On the master, it serves as the source of map outputs recorded from ShuffleMapTasks. + * On the workers, it simply serves as a cache, in which a miss triggers a fetch from the + * master's corresponding HashMap. + */ + protected val mapStatuses: Map[Int, Array[MapStatus]] - // Incremented every time a fetch fails so that client nodes know to clear - // their cache of map output locations if this happens. + /** + * Incremented every time a fetch fails so that client nodes know to clear + * their cache of map output locations if this happens. + */ protected var epoch: Long = 0 - protected val epochLock = new java.lang.Object + protected val epochLock = new AnyRef - private val metadataCleaner = - new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup, conf) + /** Remembers which map output locations are currently being fetched on a worker. */ + private val fetching = new HashSet[Int] - // Send a message to the trackerActor and get its result within a default timeout, or - // throw a SparkException if this fails. - private def askTracker(message: Any): Any = { + /** + * Send a message to the trackerActor and get its result within a default timeout, or + * throw a SparkException if this fails. + */ + protected def askTracker(message: Any): Any = { try { val future = trackerActor.ask(message)(timeout) Await.result(future, timeout) @@ -94,17 +109,17 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { } } - // Send a one-way message to the trackerActor, to which we expect it to reply with true. - private def communicate(message: Any) { + /** Send a one-way message to the trackerActor, to which we expect it to reply with true. */ + protected def sendTracker(message: Any) { if (askTracker(message) != true) { throw new SparkException("Error reply received from MapOutputTracker") } } - // Remembers which map output locations are currently being fetched on a worker - private val fetching = new HashSet[Int] - - // Called on possibly remote nodes to get the server URIs and output sizes for a given shuffle + /** + * Called from executors to get the server URIs and output sizes of the map outputs of + * a given shuffle. + */ def getServerStatuses(shuffleId: Int, reduceId: Int): Array[(BlockManagerId, Long)] = { val statuses = mapStatuses.get(shuffleId).orNull if (statuses == null) { @@ -152,8 +167,7 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { fetchedStatuses.synchronized { return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, fetchedStatuses) } - } - else { + } else { throw new FetchFailedException(null, shuffleId, -1, reduceId, new Exception("Missing all output locations for shuffle " + shuffleId)) } @@ -164,27 +178,18 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { } } - protected def cleanup(cleanupTime: Long) { - mapStatuses.clearOldValues(cleanupTime) - } - - def stop() { - communicate(StopMapOutputTracker) - mapStatuses.clear() - metadataCleaner.cancel() - trackerActor = null - } - - // Called to get current epoch number + /** Called to get current epoch number. */ def getEpoch: Long = { epochLock.synchronized { return epoch } } - // Called on workers to update the epoch number, potentially clearing old outputs - // because of a fetch failure. (Each worker task calls this with the latest epoch - // number on the master at the time it was created.) + /** + * Called from executors to update the epoch number, potentially clearing old outputs + * because of a fetch failure. Each worker task calls this with the latest epoch + * number on the master at the time it was created. + */ def updateEpoch(newEpoch: Long) { epochLock.synchronized { if (newEpoch > epoch) { @@ -194,17 +199,40 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { } } } + + /** Unregister shuffle data. */ + def unregisterShuffle(shuffleId: Int) { + mapStatuses.remove(shuffleId) + } + + /** Stop the tracker. */ + def stop() { } } +/** + * MapOutputTracker for the driver. This uses TimeStampedHashMap to keep track of map + * output information, which allows old output information based on a TTL. + */ private[spark] class MapOutputTrackerMaster(conf: SparkConf) extends MapOutputTracker(conf) { - // Cache a serialized version of the output statuses for each shuffle to send them out faster + /** Cache a serialized version of the output statuses for each shuffle to send them out faster */ private var cacheEpoch = epoch - private val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]] + + /** + * Timestamp based HashMap for storing mapStatuses and cached serialized statuses in the master, + * so that statuses are dropped only by explicit de-registering or by TTL-based cleaning (if set). + * Other than these two scenarios, nothing should be dropped from this HashMap. + */ + protected val mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]() + private val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]]() + + // For cleaning up TimeStampedHashMaps + private val metadataCleaner = + new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup, conf) def registerShuffle(shuffleId: Int, numMaps: Int) { - if (mapStatuses.putIfAbsent(shuffleId, new Array[MapStatus](numMaps)).isDefined) { + if (mapStatuses.put(shuffleId, new Array[MapStatus](numMaps)).isDefined) { throw new IllegalArgumentException("Shuffle ID " + shuffleId + " registered twice") } } @@ -216,6 +244,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) } } + /** Register multiple map output information for the given shuffle */ def registerMapOutputs(shuffleId: Int, statuses: Array[MapStatus], changeEpoch: Boolean = false) { mapStatuses.put(shuffleId, Array[MapStatus]() ++ statuses) if (changeEpoch) { @@ -223,6 +252,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) } } + /** Unregister map output information of the given shuffle, mapper and block manager */ def unregisterMapOutput(shuffleId: Int, mapId: Int, bmAddress: BlockManagerId) { val arrayOpt = mapStatuses.get(shuffleId) if (arrayOpt.isDefined && arrayOpt.get != null) { @@ -238,6 +268,17 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) } } + /** Unregister shuffle data */ + override def unregisterShuffle(shuffleId: Int) { + mapStatuses.remove(shuffleId) + cachedSerializedStatuses.remove(shuffleId) + } + + /** Check if the given shuffle is being tracked */ + def containsShuffle(shuffleId: Int): Boolean = { + cachedSerializedStatuses.contains(shuffleId) || mapStatuses.contains(shuffleId) + } + def incrementEpoch() { epochLock.synchronized { epoch += 1 @@ -274,23 +315,26 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) bytes } - protected override def cleanup(cleanupTime: Long) { - super.cleanup(cleanupTime) - cachedSerializedStatuses.clearOldValues(cleanupTime) - } - override def stop() { - super.stop() + sendTracker(StopMapOutputTracker) + mapStatuses.clear() + trackerActor = null + metadataCleaner.cancel() cachedSerializedStatuses.clear() } - override def updateEpoch(newEpoch: Long) { - // This might be called on the MapOutputTrackerMaster if we're running in local mode. + private def cleanup(cleanupTime: Long) { + mapStatuses.clearOldValues(cleanupTime) + cachedSerializedStatuses.clearOldValues(cleanupTime) } +} - def has(shuffleId: Int): Boolean = { - cachedSerializedStatuses.get(shuffleId).isDefined || mapStatuses.contains(shuffleId) - } +/** + * MapOutputTracker for the workers, which fetches map output information from the driver's + * MapOutputTrackerMaster. + */ +private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTracker(conf) { + protected val mapStatuses = new HashMap[Int, Array[MapStatus]] } private[spark] object MapOutputTracker { diff --git a/core/src/main/scala/org/apache/spark/Partition.scala b/core/src/main/scala/org/apache/spark/Partition.scala index 87914a061f5d7..27892dbd2a0bc 100644 --- a/core/src/main/scala/org/apache/spark/Partition.scala +++ b/core/src/main/scala/org/apache/spark/Partition.scala @@ -25,7 +25,7 @@ trait Partition extends Serializable { * Get the split's index within its parent RDD */ def index: Int - + // A better default implementation of HashCode override def hashCode(): Int = index } diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index ad9988226470c..e7f75481939a8 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -83,18 +83,26 @@ class HashPartitioner(partitions: Int) extends Partitioner { case _ => false } + + override def hashCode: Int = numPartitions } /** * A [[org.apache.spark.Partitioner]] that partitions sortable records by range into roughly * equal ranges. The ranges are determined by sampling the content of the RDD passed in. + * + * Note that the actual number of partitions created by the RangePartitioner might not be the same + * as the `partitions` parameter, in the case where the number of sampled records is less than + * the value of `partitions`. */ -class RangePartitioner[K <% Ordered[K]: ClassTag, V]( +class RangePartitioner[K : Ordering : ClassTag, V]( partitions: Int, @transient rdd: RDD[_ <: Product2[K,V]], private val ascending: Boolean = true) extends Partitioner { + private val ordering = implicitly[Ordering[K]] + // An array of upper bounds for the first (partitions - 1) partitions private val rangeBounds: Array[K] = { if (partitions == 1) { @@ -103,7 +111,7 @@ class RangePartitioner[K <% Ordered[K]: ClassTag, V]( val rddSize = rdd.count() val maxSampleSize = partitions * 20.0 val frac = math.min(maxSampleSize / math.max(rddSize, 1), 1.0) - val rddSample = rdd.sample(false, frac, 1).map(_._1).collect().sortWith(_ < _) + val rddSample = rdd.sample(false, frac, 1).map(_._1).collect().sorted if (rddSample.length == 0) { Array() } else { @@ -117,7 +125,7 @@ class RangePartitioner[K <% Ordered[K]: ClassTag, V]( } } - def numPartitions = partitions + def numPartitions = rangeBounds.length + 1 private val binarySearch: ((Array[K], K) => Int) = CollectionsUtils.makeBinarySearch[K] @@ -126,7 +134,7 @@ class RangePartitioner[K <% Ordered[K]: ClassTag, V]( var partition = 0 if (rangeBounds.length < 1000) { // If we have less than 100 partitions naive search - while (partition < rangeBounds.length && k > rangeBounds(partition)) { + while (partition < rangeBounds.length && ordering.gt(k, rangeBounds(partition))) { partition += 1 } } else { @@ -153,4 +161,16 @@ class RangePartitioner[K <% Ordered[K]: ClassTag, V]( case _ => false } + + override def hashCode(): Int = { + val prime = 31 + var result = 1 + var i = 0 + while (i < rangeBounds.length) { + result = prime * result + rangeBounds(i).hashCode + i += 1 + } + result = prime * result + ascending.hashCode + result + } } diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 2237ee3bb7aad..74aa441619bd2 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -19,99 +19,97 @@ package org.apache.spark import java.net.{Authenticator, PasswordAuthentication} -import scala.collection.mutable.ArrayBuffer - import org.apache.hadoop.io.Text import org.apache.spark.deploy.SparkHadoopUtil -/** - * Spark class responsible for security. - * +/** + * Spark class responsible for security. + * * In general this class should be instantiated by the SparkEnv and most components - * should access it from that. There are some cases where the SparkEnv hasn't been + * should access it from that. There are some cases where the SparkEnv hasn't been * initialized yet and this class must be instantiated directly. - * + * * Spark currently supports authentication via a shared secret. * Authentication can be configured to be on via the 'spark.authenticate' configuration - * parameter. This parameter controls whether the Spark communication protocols do + * parameter. This parameter controls whether the Spark communication protocols do * authentication using the shared secret. This authentication is a basic handshake to * make sure both sides have the same shared secret and are allowed to communicate. - * If the shared secret is not identical they will not be allowed to communicate. - * - * The Spark UI can also be secured by using javax servlet filters. A user may want to - * secure the UI if it has data that other users should not be allowed to see. The javax - * servlet filter specified by the user can authenticate the user and then once the user - * is logged in, Spark can compare that user versus the view acls to make sure they are - * authorized to view the UI. The configs 'spark.ui.acls.enable' and 'spark.ui.view.acls' + * If the shared secret is not identical they will not be allowed to communicate. + * + * The Spark UI can also be secured by using javax servlet filters. A user may want to + * secure the UI if it has data that other users should not be allowed to see. The javax + * servlet filter specified by the user can authenticate the user and then once the user + * is logged in, Spark can compare that user versus the view acls to make sure they are + * authorized to view the UI. The configs 'spark.ui.acls.enable' and 'spark.ui.view.acls' * control the behavior of the acls. Note that the person who started the application * always has view access to the UI. * * Spark does not currently support encryption after authentication. - * + * * At this point spark has multiple communication protocols that need to be secured and * different underlying mechanisms are used depending on the protocol: * - * - Akka -> The only option here is to use the Akka Remote secure-cookie functionality. - * Akka remoting allows you to specify a secure cookie that will be exchanged - * and ensured to be identical in the connection handshake between the client - * and the server. If they are not identical then the client will be refused - * to connect to the server. There is no control of the underlying - * authentication mechanism so its not clear if the password is passed in + * - Akka -> The only option here is to use the Akka Remote secure-cookie functionality. + * Akka remoting allows you to specify a secure cookie that will be exchanged + * and ensured to be identical in the connection handshake between the client + * and the server. If they are not identical then the client will be refused + * to connect to the server. There is no control of the underlying + * authentication mechanism so its not clear if the password is passed in * plaintext or uses DIGEST-MD5 or some other mechanism. * Akka also has an option to turn on SSL, this option is not currently supported * but we could add a configuration option in the future. - * - * - HTTP for broadcast and file server (via HttpServer) -> Spark currently uses Jetty - * for the HttpServer. Jetty supports multiple authentication mechanisms - - * Basic, Digest, Form, Spengo, etc. It also supports multiple different login + * + * - HTTP for broadcast and file server (via HttpServer) -> Spark currently uses Jetty + * for the HttpServer. Jetty supports multiple authentication mechanisms - + * Basic, Digest, Form, Spengo, etc. It also supports multiple different login * services - Hash, JAAS, Spnego, JDBC, etc. Spark currently uses the HashLoginService - * to authenticate using DIGEST-MD5 via a single user and the shared secret. + * to authenticate using DIGEST-MD5 via a single user and the shared secret. * Since we are using DIGEST-MD5, the shared secret is not passed on the wire * in plaintext. * We currently do not support SSL (https), but Jetty can be configured to use it * so we could add a configuration option for this in the future. - * + * * The Spark HttpServer installs the HashLoginServer and configures it to DIGEST-MD5. - * Any clients must specify the user and password. There is a default + * Any clients must specify the user and password. There is a default * Authenticator installed in the SecurityManager to how it does the authentication * and in this case gets the user name and password from the request. * - * - ConnectionManager -> The Spark ConnectionManager uses java nio to asynchronously - * exchange messages. For this we use the Java SASL - * (Simple Authentication and Security Layer) API and again use DIGEST-MD5 + * - ConnectionManager -> The Spark ConnectionManager uses java nio to asynchronously + * exchange messages. For this we use the Java SASL + * (Simple Authentication and Security Layer) API and again use DIGEST-MD5 * as the authentication mechanism. This means the shared secret is not passed * over the wire in plaintext. * Note that SASL is pluggable as to what mechanism it uses. We currently use * DIGEST-MD5 but this could be changed to use Kerberos or other in the future. * Spark currently supports "auth" for the quality of protection, which means * the connection is not supporting integrity or privacy protection (encryption) - * after authentication. SASL also supports "auth-int" and "auth-conf" which + * after authentication. SASL also supports "auth-int" and "auth-conf" which * SPARK could be support in the future to allow the user to specify the quality - * of protection they want. If we support those, the messages will also have to + * of protection they want. If we support those, the messages will also have to * be wrapped and unwrapped via the SaslServer/SaslClient.wrap/unwrap API's. - * - * Since the connectionManager does asynchronous messages passing, the SASL + * + * Since the connectionManager does asynchronous messages passing, the SASL * authentication is a bit more complex. A ConnectionManager can be both a client * and a Server, so for a particular connection is has to determine what to do. - * A ConnectionId was added to be able to track connections and is used to + * A ConnectionId was added to be able to track connections and is used to * match up incoming messages with connections waiting for authentication. * If its acting as a client and trying to send a message to another ConnectionManager, * it blocks the thread calling sendMessage until the SASL negotiation has occurred. * The ConnectionManager tracks all the sendingConnections using the ConnectionId * and waits for the response from the server and does the handshake. * - * - HTTP for the Spark UI -> the UI was changed to use servlets so that javax servlet filters + * - HTTP for the Spark UI -> the UI was changed to use servlets so that javax servlet filters * can be used. Yarn requires a specific AmIpFilter be installed for security to work * properly. For non-Yarn deployments, users can write a filter to go through a * companies normal login service. If an authentication filter is in place then the * SparkUI can be configured to check the logged in user against the list of users who * have view acls to see if that user is authorized. - * The filters can also be used for many different purposes. For instance filters + * The filters can also be used for many different purposes. For instance filters * could be used for logging, encryption, or compression. - * + * * The exact mechanisms used to generate/distributed the shared secret is deployment specific. - * + * * For Yarn deployments, the secret is automatically generated using the Akka remote * Crypt.generateSecureCookie() API. The secret is placed in the Hadoop UGI which gets passed * around via the Hadoop RPC mechanism. Hadoop RPC can be configured to support different levels @@ -121,7 +119,7 @@ import org.apache.spark.deploy.SparkHadoopUtil * to reduce the possibility of web based attacks through YARN. Hadoop can be configured to use * filters to do authentication. That authentication then happens via the ResourceManager Proxy * and Spark will use that to do authorization against the view acls. - * + * * For other Spark deployments, the shared secret must be specified via the * spark.authenticate.secret config. * All the nodes (Master and Workers) and the applications need to have the same shared secret. @@ -139,20 +137,21 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { private val sparkSecretLookupKey = "sparkCookie" private val authOn = sparkConf.getBoolean("spark.authenticate", false) - private val uiAclsOn = sparkConf.getBoolean("spark.ui.acls.enable", false) + private var uiAclsOn = sparkConf.getBoolean("spark.ui.acls.enable", false) + private var viewAcls: Set[String] = _ // always add the current user and SPARK_USER to the viewAcls - private val aclUsers = ArrayBuffer[String](System.getProperty("user.name", ""), + private val defaultAclUsers = Seq[String](System.getProperty("user.name", ""), Option(System.getenv("SPARK_USER")).getOrElse("")) - aclUsers ++= sparkConf.get("spark.ui.view.acls", "").split(',') - private val viewAcls = aclUsers.map(_.trim()).filter(!_.isEmpty).toSet + setViewAcls(defaultAclUsers, sparkConf.get("spark.ui.view.acls", "")) private val secretKey = generateSecretKey() - logInfo("SecurityManager, is authentication enabled: " + authOn + - " are ui acls enabled: " + uiAclsOn + " users with view permissions: " + viewAcls.toString()) + logInfo("SecurityManager: authentication " + (if (authOn) "enabled" else "disabled") + + "; ui acls " + (if (uiAclsOn) "enabled" else "disabled") + + "; users with view permissions: " + viewAcls.toString()) // Set our own authenticator to properly negotiate user/password for HTTP connections. - // This is needed by the HTTP client fetching from the HttpServer. Put here so its + // This is needed by the HTTP client fetching from the HttpServer. Put here so its // only set once. if (authOn) { Authenticator.setDefault( @@ -170,6 +169,20 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { ) } + private[spark] def setViewAcls(defaultUsers: Seq[String], allowedUsers: String) { + viewAcls = (defaultUsers ++ allowedUsers.split(',')).map(_.trim()).filter(!_.isEmpty).toSet + logInfo("Changing view acls to: " + viewAcls.mkString(",")) + } + + private[spark] def setViewAcls(defaultUser: String, allowedUsers: String) { + setViewAcls(Seq[String](defaultUser), allowedUsers) + } + + private[spark] def setUIAcls(aclSetting: Boolean) { + uiAclsOn = aclSetting + logInfo("Changing acls enabled to: " + uiAclsOn) + } + /** * Generates or looks up the secret key. * @@ -214,14 +227,16 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { def uiAclsEnabled(): Boolean = uiAclsOn /** - * Checks the given user against the view acl list to see if they have + * Checks the given user against the view acl list to see if they have * authorization to view the UI. If the UI acls must are disabled * via spark.ui.acls.enable, all users have view access. - * + * * @param user to see if is authorized - * @return true is the user has permission, otherwise false + * @return true is the user has permission, otherwise false */ def checkUIViewPermissions(user: String): Boolean = { + logDebug("user=" + user + " uiAclsEnabled=" + uiAclsEnabled() + " viewAcls=" + + viewAcls.mkString(",")) if (uiAclsEnabled() && (user != null) && (!viewAcls.contains(user))) false else true } diff --git a/core/src/main/scala/org/apache/spark/SerializableWritable.scala b/core/src/main/scala/org/apache/spark/SerializableWritable.scala index dff665cae6cb6..e50b9ac2291f9 100644 --- a/core/src/main/scala/org/apache/spark/SerializableWritable.scala +++ b/core/src/main/scala/org/apache/spark/SerializableWritable.scala @@ -23,6 +23,9 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.ObjectWritable import org.apache.hadoop.io.Writable +import org.apache.spark.annotation.DeveloperApi + +@DeveloperApi class SerializableWritable[T <: Writable](@transient var t: T) extends Serializable { def value = t override def toString = t.toString diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index b947feb891ee6..8ce4b91cae8ae 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -208,6 +208,82 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { new SparkConf(false).setAll(settings) } + /** Checks for illegal or deprecated config settings. Throws an exception for the former. Not + * idempotent - may mutate this conf object to convert deprecated settings to supported ones. */ + private[spark] def validateSettings() { + if (settings.contains("spark.local.dir")) { + val msg = "In Spark 1.0 and later spark.local.dir will be overridden by the value set by " + + "the cluster manager (via SPARK_LOCAL_DIRS in mesos/standalone and LOCAL_DIRS in YARN)." + logWarning(msg) + } + + val executorOptsKey = "spark.executor.extraJavaOptions" + val executorClasspathKey = "spark.executor.extraClassPath" + val driverOptsKey = "spark.driver.extraJavaOptions" + val driverClassPathKey = "spark.driver.extraClassPath" + + // Validate spark.executor.extraJavaOptions + settings.get(executorOptsKey).map { javaOpts => + if (javaOpts.contains("-Dspark")) { + val msg = s"$executorOptsKey is not allowed to set Spark options (was '$javaOpts)'. " + + "Set them directly on a SparkConf or in a properties file when using ./bin/spark-submit." + throw new Exception(msg) + } + if (javaOpts.contains("-Xmx") || javaOpts.contains("-Xms")) { + val msg = s"$executorOptsKey is not allowed to alter memory settings (was '$javaOpts'). " + + "Use spark.executor.memory instead." + throw new Exception(msg) + } + } + + // Check for legacy configs + sys.env.get("SPARK_JAVA_OPTS").foreach { value => + val warning = + s""" + |SPARK_JAVA_OPTS was detected (set to '$value'). + |This is deprecated in Spark 1.0+. + | + |Please instead use: + | - ./spark-submit with conf/spark-defaults.conf to set defaults for an application + | - ./spark-submit with --driver-java-options to set -X options for a driver + | - spark.executor.extraJavaOptions to set -X options for executors + | - SPARK_DAEMON_JAVA_OPTS to set java options for standalone daemons (master or worker) + """.stripMargin + logWarning(warning) + + for (key <- Seq(executorOptsKey, driverOptsKey)) { + if (getOption(key).isDefined) { + throw new SparkException(s"Found both $key and SPARK_JAVA_OPTS. Use only the former.") + } else { + logWarning(s"Setting '$key' to '$value' as a work-around.") + set(key, value) + } + } + } + + sys.env.get("SPARK_CLASSPATH").foreach { value => + val warning = + s""" + |SPARK_CLASSPATH was detected (set to '$value'). + |This is deprecated in Spark 1.0+. + | + |Please instead use: + | - ./spark-submit with --driver-class-path to augment the driver classpath + | - spark.executor.extraClassPath to augment the executor classpath + """.stripMargin + logWarning(warning) + + for (key <- Seq(executorClasspathKey, driverClassPathKey)) { + if (getOption(key).isDefined) { + throw new SparkException(s"Found both $key and SPARK_CLASSPATH. Use only the former.") + } else { + logWarning(s"Setting '$key' to '$value' as a work-around.") + set(key, value) + } + } + } + } + /** * Return a string listing all keys and values, one per line. This is useful to print the * configuration out for debugging. diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index fcf16ce1b278e..f9476ff826a62 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -17,14 +17,17 @@ package org.apache.spark +import scala.language.implicitConversions + import java.io._ import java.net.URI import java.util.concurrent.atomic.AtomicInteger import java.util.{Properties, UUID} import java.util.UUID.randomUUID import scala.collection.{Map, Set} +import scala.collection.JavaConversions._ import scala.collection.generic.Growable -import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.mutable.HashMap import scala.reflect.{ClassTag, classTag} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -34,6 +37,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHad import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} 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.input.WholeTextFileInputFormat @@ -45,7 +49,7 @@ import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, Me import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils} import org.apache.spark.ui.SparkUI -import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} +import org.apache.spark.util.{CallSite, ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -53,17 +57,34 @@ import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerTy * * @param config a Spark Config object describing the application configuration. Any settings in * this config overrides the default configs as well as system properties. - * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Can - * be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] - * from a list of input files or InputFormats for the application. */ -class SparkContext( - config: SparkConf, - // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, - // etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It - // contains a map from hostname to a list of input format splits on the host. - val preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) - extends Logging { + +class SparkContext(config: SparkConf) extends Logging { + + // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, + // etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It + // contains a map from hostname to a list of input format splits on the host. + private[spark] var preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map() + + /** + * Create a SparkContext that loads settings from system properties (for instance, when + * launching with ./bin/spark-submit). + */ + def this() = this(new SparkConf()) + + /** + * :: DeveloperApi :: + * Alternative constructor for setting preferred locations where Spark will create executors. + * + * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. + * Can be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] + * from a list of input files or InputFormats for the application. + */ + @DeveloperApi + def this(config: SparkConf, preferredNodeLocationData: Map[String, Set[SplitInfo]]) = { + this(config) + this.preferredNodeLocationData = preferredNodeLocationData + } /** * Alternative constructor that allows setting common Spark properties directly @@ -93,11 +114,47 @@ class SparkContext( environment: Map[String, String] = Map(), preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) = { - this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment), - preferredNodeLocationData) + this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment)) + this.preferredNodeLocationData = preferredNodeLocationData } + // NOTE: The below constructors could be consolidated using default arguments. Due to + // Scala bug SI-8479, however, this causes the compile step to fail when generating docs. + // Until we have a good workaround for that bug the constructors remain broken out. + + /** + * Alternative constructor that allows setting common Spark properties directly + * + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI. + */ + private[spark] def this(master: String, appName: String) = + this(master, appName, null, Nil, Map(), Map()) + + /** + * Alternative constructor that allows setting common Spark properties directly + * + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI. + * @param sparkHome Location where Spark is installed on cluster nodes. + */ + private[spark] def this(master: String, appName: String, sparkHome: String) = + this(master, appName, sparkHome, Nil, Map(), Map()) + + /** + * Alternative constructor that allows setting common Spark properties directly + * + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI. + * @param sparkHome Location where Spark is installed on cluster nodes. + * @param jars Collection of JARs to send to the cluster. These can be paths on the local file + * system or HDFS, HTTP, HTTPS, or FTP URLs. + */ + private[spark] def this(master: String, appName: String, sparkHome: String, jars: Seq[String]) = + this(master, appName, sparkHome, jars, Map(), Map()) + private[spark] val conf = config.clone() + conf.validateSettings() /** * Return a copy of this SparkContext's configuration. The configuration ''cannot'' be @@ -109,7 +166,7 @@ class SparkContext( throw new SparkException("A master URL must be set in your configuration") } if (!conf.contains("spark.app.name")) { - throw new SparkException("An application must be set in your configuration") + throw new SparkException("An application name must be set in your configuration") } if (conf.getBoolean("spark.logConf", false)) { @@ -120,11 +177,11 @@ class SparkContext( conf.setIfMissing("spark.driver.host", Utils.localHostName()) conf.setIfMissing("spark.driver.port", "0") - val jars: Seq[String] = if (conf.contains("spark.jars")) { - conf.get("spark.jars").split(",").filter(_.size != 0) - } else { - null - } + 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") @@ -157,28 +214,46 @@ class SparkContext( private[spark] val addedJars = HashMap[String, Long]() // Keeps track of all persisted RDDs - private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]] + private[spark] val persistentRdds = new TimeStampedWeakValueHashMap[Int, RDD[_]] private[spark] val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf) // Initialize the Spark UI, registering all associated listeners private[spark] val ui = new SparkUI(this) ui.bind() - ui.start() + + /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ + val hadoopConfiguration: Configuration = { + val hadoopConf = SparkHadoopUtil.get.newConfiguration() + // Explicitly check for S3 environment variables + if (System.getenv("AWS_ACCESS_KEY_ID") != null && + System.getenv("AWS_SECRET_ACCESS_KEY") != null) { + hadoopConf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) + hadoopConf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) + hadoopConf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) + hadoopConf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) + } + // Copy any "spark.hadoop.foo=bar" system properties into conf as "foo=bar" + conf.getAll.foreach { case (key, value) => + if (key.startsWith("spark.hadoop.")) { + hadoopConf.set(key.substring("spark.hadoop.".length), value) + } + } + val bufferSize = conf.get("spark.buffer.size", "65536") + hadoopConf.set("io.file.buffer.size", bufferSize) + hadoopConf + } // Optionally log Spark events private[spark] val eventLogger: Option[EventLoggingListener] = { if (conf.getBoolean("spark.eventLog.enabled", false)) { - val logger = new EventLoggingListener(appName, conf) + val logger = new EventLoggingListener(appName, conf, hadoopConfiguration) + logger.start() listenerBus.addListener(logger) Some(logger) } else None } - // Information needed to replay logged events, if any - private[spark] val eventLoggingInfo: Option[EventLoggingInfo] = - eventLogger.map { logger => Some(logger.info) }.getOrElse(None) - // At this point, all relevant SparkListeners have been registered, so begin releasing events listenerBus.start() @@ -189,7 +264,11 @@ class SparkContext( jars.foreach(addJar) } - def warnSparkMem(value: String): String = { + if (files != null) { + files.foreach(addFile) + } + + 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 @@ -201,18 +280,18 @@ class SparkContext( .map(Utils.memoryStringToMb) .getOrElse(512) - // Environment variables to pass to our executors + // Environment variables to pass to our executors. private[spark] val executorEnvs = HashMap[String, String]() - for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS"); - value <- Option(System.getenv(key))) { - executorEnvs(key) = value - } + // 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_HOME", "spark.home"), ("SPARK_TESTING", "spark.testing")) + 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" @@ -220,7 +299,7 @@ class SparkContext( // Set SPARK_USER for user who is running SparkContext. val sparkUser = Option { - Option(System.getProperty("user.name")).getOrElse(System.getenv("SPARK_USER")) + Option(System.getenv("SPARK_USER")).getOrElse(System.getProperty("user.name")) }.getOrElse { SparkContext.SPARK_UNKNOWN_USER } @@ -228,35 +307,29 @@ class SparkContext( // Create and start the scheduler private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master) - taskScheduler.start() - - @volatile private[spark] var dagScheduler = new DAGScheduler(this) - dagScheduler.start() + @volatile private[spark] var dagScheduler: DAGScheduler = _ + try { + dagScheduler = new DAGScheduler(this) + } catch { + case e: Exception => throw + new SparkException("DAGScheduler cannot be initialized due to %s".format(e.getMessage)) + } - postEnvironmentUpdate() + // start TaskScheduler after taskScheduler sets DAGScheduler reference in DAGScheduler's + // constructor + taskScheduler.start() - /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ - val hadoopConfiguration: Configuration = { - val env = SparkEnv.get - val hadoopConf = SparkHadoopUtil.get.newConfiguration() - // Explicitly check for S3 environment variables - if (System.getenv("AWS_ACCESS_KEY_ID") != null && - System.getenv("AWS_SECRET_ACCESS_KEY") != null) { - hadoopConf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) - hadoopConf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) - hadoopConf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) - hadoopConf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) - } - // Copy any "spark.hadoop.foo=bar" system properties into conf as "foo=bar" - conf.getAll.foreach { case (key, value) => - if (key.startsWith("spark.hadoop.")) { - hadoopConf.set(key.substring("spark.hadoop.".length), value) - } + private[spark] val cleaner: Option[ContextCleaner] = { + if (conf.getBoolean("spark.cleaner.referenceTracking", true)) { + Some(new ContextCleaner(this)) + } else { + None } - val bufferSize = conf.get("spark.buffer.size", "65536") - hadoopConf.set("io.file.buffer.size", bufferSize) - hadoopConf } + cleaner.foreach(_.start()) + + postEnvironmentUpdate() + postApplicationStart() private[spark] var checkpointDir: Option[String] = None @@ -322,16 +395,27 @@ class SparkContext( * // In a separate thread: * sc.cancelJobGroup("some_job_to_cancel") * }}} + * + * If interruptOnCancel is set to true for the job group, then job cancellation will result + * in Thread.interrupt() being called on the job's executor threads. This is useful to help ensure + * that the tasks are actually stopped in a timely manner, but is off by default due to HDFS-1208, + * where HDFS may respond to Thread.interrupt() by marking nodes as dead. */ - def setJobGroup(groupId: String, description: String) { + def setJobGroup(groupId: String, description: String, interruptOnCancel: Boolean = false) { setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, description) setLocalProperty(SparkContext.SPARK_JOB_GROUP_ID, groupId) + // Note: Specifying interruptOnCancel in setJobGroup (rather than cancelJobGroup) avoids + // changing several public APIs and allows Spark cancellations outside of the cancelJobGroup + // APIs to also take advantage of this property (e.g., internal job failures or canceling from + // JobProgressTab UI) on a per-job basis. + setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, interruptOnCancel.toString) } /** Clear the current thread's job group ID and its description. */ def clearJobGroup() { setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, null) setLocalProperty(SparkContext.SPARK_JOB_GROUP_ID, null) + setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, null) } // Post init @@ -349,12 +433,21 @@ class SparkContext( // Methods for creating RDDs - /** Distribute a local Scala collection to form an RDD. */ + /** Distribute a local Scala collection to form an RDD. + * + * @note Parallelize acts lazily. If `seq` is a mutable collection and is + * altered after the call to parallelize and before the first action on the + * RDD, the resultant RDD will reflect the modified collection. Pass a copy of + * the argument to avoid this. + */ def parallelize[T: ClassTag](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = { new ParallelCollectionRDD[T](this, seq, numSlices, Map[Int, Seq[String]]()) } - /** Distribute a local Scala collection to form an RDD. */ + /** Distribute a local Scala collection to form an RDD. + * + * This method is identical to `parallelize`. + */ def makeRDD[T: ClassTag](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = { parallelize(seq, numSlices) } @@ -371,9 +464,9 @@ class SparkContext( * 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, minSplits: Int = defaultMinSplits): RDD[String] = { + def textFile(path: String, minPartitions: Int = defaultMinPartitions): RDD[String] = { hadoopFile(path, classOf[TextInputFormat], classOf[LongWritable], classOf[Text], - minSplits).map(pair => pair._2.toString) + minPartitions).map(pair => pair._2.toString).setName(path) } /** @@ -399,14 +492,22 @@ class SparkContext( * (a-hdfs-path/part-nnnnn, its content) * }}} * - * @note Small files are preferred, as each file will be loaded fully in memory. + * @note Small files are preferred, large file is also allowable, but may cause bad performance. + * + * @param minPartitions A suggestion value of the minimal splitting number for input data. */ - def wholeTextFiles(path: String): RDD[(String, String)] = { - newAPIHadoopFile( - path, + def wholeTextFiles(path: String, minPartitions: Int = defaultMinPartitions): + RDD[(String, String)] = { + val job = new NewHadoopJob(hadoopConfiguration) + NewFileInputFormat.addInputPath(job, new Path(path)) + val updateConf = job.getConfiguration + new WholeTextFileRDD( + this, classOf[WholeTextFileInputFormat], classOf[String], - classOf[String]) + classOf[String], + updateConf, + minPartitions).setName(path) } /** @@ -418,7 +519,7 @@ class SparkContext( * @param inputFormatClass Class of the InputFormat * @param keyClass Class of the keys * @param valueClass Class of the values - * @param minSplits Minimum number of Hadoop Splits to generate. + * @param minPartitions Minimum number of Hadoop Splits to generate. * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each * record, directly caching the returned RDD will create many references to the same object. @@ -430,11 +531,11 @@ class SparkContext( inputFormatClass: Class[_ <: InputFormat[K, V]], keyClass: Class[K], valueClass: Class[V], - minSplits: Int = defaultMinSplits + minPartitions: Int = defaultMinPartitions ): RDD[(K, V)] = { // Add necessary security credentials to the JobConf before broadcasting it. SparkHadoopUtil.get.addCredentials(conf) - new HadoopRDD(this, conf, inputFormatClass, keyClass, valueClass, minSplits) + new HadoopRDD(this, conf, inputFormatClass, keyClass, valueClass, minPartitions) } /** Get an RDD for a Hadoop file with an arbitrary InputFormat @@ -449,7 +550,7 @@ class SparkContext( inputFormatClass: Class[_ <: InputFormat[K, V]], keyClass: Class[K], valueClass: Class[V], - minSplits: Int = defaultMinSplits + minPartitions: Int = defaultMinPartitions ): RDD[(K, V)] = { // A Hadoop configuration can be about 10 KB, which is pretty big, so broadcast it. val confBroadcast = broadcast(new SerializableWritable(hadoopConfiguration)) @@ -461,7 +562,7 @@ class SparkContext( inputFormatClass, keyClass, valueClass, - minSplits) + minPartitions).setName(path) } /** @@ -469,7 +570,7 @@ class SparkContext( * values and the InputFormat so that users don't need to pass them directly. Instead, callers * can just write, for example, * {{{ - * val file = sparkContext.hadoopFile[LongWritable, Text, TextInputFormat](path, minSplits) + * val file = sparkContext.hadoopFile[LongWritable, Text, TextInputFormat](path, minPartitions) * }}} * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each @@ -478,13 +579,13 @@ class SparkContext( * a `map` function. */ def hadoopFile[K, V, F <: InputFormat[K, V]] - (path: String, minSplits: Int) + (path: String, minPartitions: Int) (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = { hadoopFile(path, fm.runtimeClass.asInstanceOf[Class[F]], km.runtimeClass.asInstanceOf[Class[K]], vm.runtimeClass.asInstanceOf[Class[V]], - minSplits) + minPartitions) } /** @@ -502,7 +603,7 @@ class SparkContext( */ def hadoopFile[K, V, F <: InputFormat[K, V]](path: String) (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = - hadoopFile[K, V, F](path, defaultMinSplits) + 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]] @@ -533,7 +634,7 @@ class SparkContext( val job = new NewHadoopJob(conf) NewFileInputFormat.addInputPath(job, new Path(path)) val updatedConf = job.getConfiguration - new NewHadoopRDD(this, fClass, kClass, vClass, updatedConf) + new NewHadoopRDD(this, fClass, kClass, vClass, updatedConf).setName(path) } /** @@ -563,10 +664,10 @@ class SparkContext( def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V], - minSplits: Int + minPartitions: Int ): RDD[(K, V)] = { val inputFormatClass = classOf[SequenceFileInputFormat[K, V]] - hadoopFile(path, inputFormatClass, keyClass, valueClass, minSplits) + hadoopFile(path, inputFormatClass, keyClass, valueClass, minPartitions) } /** Get an RDD for a Hadoop SequenceFile with given key and value types. @@ -578,7 +679,7 @@ class SparkContext( * */ def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V] ): RDD[(K, V)] = - sequenceFile(path, keyClass, valueClass, defaultMinSplits) + sequenceFile(path, keyClass, valueClass, defaultMinPartitions) /** * Version of sequenceFile() for types implicitly convertible to Writables through a @@ -602,7 +703,7 @@ class SparkContext( * a `map` function. */ def sequenceFile[K, V] - (path: String, minSplits: Int = defaultMinSplits) + (path: String, minPartitions: Int = defaultMinPartitions) (implicit km: ClassTag[K], vm: ClassTag[V], kcf: () => WritableConverter[K], vcf: () => WritableConverter[V]) : RDD[(K, V)] = { @@ -611,7 +712,7 @@ class SparkContext( val format = classOf[SequenceFileInputFormat[Writable, Writable]] val writables = hadoopFile(path, format, kc.writableClass(km).asInstanceOf[Class[Writable]], - vc.writableClass(vm).asInstanceOf[Class[Writable]], minSplits) + vc.writableClass(vm).asInstanceOf[Class[Writable]], minPartitions) writables.map { case (k, v) => (kc.convert(k), vc.convert(v)) } } @@ -625,10 +726,10 @@ class SparkContext( */ def objectFile[T: ClassTag]( path: String, - minSplits: Int = defaultMinSplits + minPartitions: Int = defaultMinPartitions ): RDD[T] = { - sequenceFile(path, classOf[NullWritable], classOf[BytesWritable], minSplits) - .flatMap(x => Utils.deserialize[Array[T]](x._2.getBytes)) + sequenceFile(path, classOf[NullWritable], classOf[BytesWritable], minPartitions) + .flatMap(x => Utils.deserialize[Array[T]](x._2.getBytes, Utils.getContextOrSparkClassLoader)) } protected[spark] def checkpointFile[T: ClassTag]( @@ -644,6 +745,9 @@ class SparkContext( def union[T: ClassTag](first: RDD[T], rest: RDD[T]*): RDD[T] = new UnionRDD(this, Seq(first) ++ rest) + /** Get an RDD that has no partitions or elements. */ + def emptyRDD[T: ClassTag] = new EmptyRDD[T](this) + // Methods for creating shared variables /** @@ -668,7 +772,7 @@ class SparkContext( * Growable and TraversableOnce are the standard APIs that guarantee += and ++=, implemented by * standard mutable collections. So you can use this with mutable Map, Set, etc. */ - def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable, T] + def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable: ClassTag, T] (initialValue: R): Accumulable[R, T] = { val param = new GrowableAccumulableParam[R,T] new Accumulable(initialValue, param) @@ -679,7 +783,11 @@ class SparkContext( * [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions. * The variable will be sent to each cluster only once. */ - def broadcast[T](value: T): Broadcast[T] = env.broadcastManager.newBroadcast[T](value, isLocal) + def broadcast[T: ClassTag](value: T): Broadcast[T] = { + val bc = env.broadcastManager.newBroadcast[T](value, isLocal) + cleaner.foreach(_.registerBroadcastForCleanup(bc)) + bc + } /** * Add a file to be downloaded with this Spark job on every node. @@ -697,16 +805,24 @@ class SparkContext( addedFiles(key) = System.currentTimeMillis // Fetch the file locally in case a job is executed using DAGScheduler.runLocally(). - Utils.fetchFile(path, new File(SparkFiles.getRootDirectory), conf, env.securityManager) + Utils.fetchFile(path, new File(SparkFiles.getRootDirectory()), conf, env.securityManager) logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key)) postEnvironmentUpdate() } + /** + * :: DeveloperApi :: + * Register a listener to receive up-calls from events that happen during execution. + */ + @DeveloperApi def addSparkListener(listener: SparkListener) { listenerBus.addListener(listener) } + /** The version of Spark on which this application is running. */ + def version = SparkContext.SPARK_VERSION + /** * Return a map from the slave to the max memory available for caching and the remaining * memory available for caching. @@ -718,9 +834,11 @@ class SparkContext( } /** + * :: DeveloperApi :: * Return information about what RDDs are cached, if they are in mem or on disk, how much space * they take, etc. */ + @DeveloperApi def getRDDStorageInfo: Array[RDDInfo] = { StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this) } @@ -731,30 +849,32 @@ class SparkContext( */ def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap - def getStageInfo: Map[Stage, StageInfo] = { - dagScheduler.stageToInfos - } - /** + * :: DeveloperApi :: * Return information about blocks stored in all of the slaves */ + @DeveloperApi def getExecutorStorageStatus: Array[StorageStatus] = { env.blockManager.master.getStorageStatus } /** - * Return pools for fair scheduler - * TODO(xiajunluan): We should take nested pools into account + * :: DeveloperApi :: + * Return pools for fair scheduler */ - def getAllPools: ArrayBuffer[Schedulable] = { - taskScheduler.rootPool.schedulableQueue + @DeveloperApi + def getAllPools: Seq[Schedulable] = { + // TODO(xiajunluan): We should take nested pools into account + taskScheduler.rootPool.schedulableQueue.toSeq } /** + * :: DeveloperApi :: * Return the pool associated with the given name, if one exists */ + @DeveloperApi def getPoolForName(pool: String): Option[Schedulable] = { - taskScheduler.rootPool.schedulableNameToSchedulable.get(pool) + Option(taskScheduler.rootPool.schedulableNameToSchedulable.get(pool)) } /** @@ -793,8 +913,7 @@ class SparkContext( /** * Unpersist an RDD from memory and/or disk storage */ - private[spark] def unpersistRDD(rdd: RDD[_], blocking: Boolean = true) { - val rddId = rdd.id + private[spark] def unpersistRDD(rddId: Int, blocking: Boolean = true) { env.blockManager.master.removeRdd(rddId, blocking) persistentRdds.remove(rddId) listenerBus.post(SparkListenerUnpersistRDD(rddId)) @@ -822,19 +941,18 @@ class SparkContext( if (SparkHadoopUtil.get.isYarnMode() && (master == "yarn-standalone" || master == "yarn-cluster")) { // In order for this to work in yarn-cluster mode the user must specify the - // --addjars option to the client to upload the file into the distributed cache + // --addJars option to the client to upload the file into the distributed cache // of the AM to make it show up in the current working directory. val fileName = new Path(uri.getPath).getName() try { env.httpFileServer.addJar(new File(fileName)) } catch { - case e: Exception => { + case e: Exception => // For now just log an error but allow to go through so spark examples work. // The spark examples don't really need the jar distributed since its also // the app jar. logError("Error adding jar (" + e + "), was the --addJars option used?") null - } } } else { env.httpFileServer.addJar(new File(uri.getPath)) @@ -865,22 +983,24 @@ class SparkContext( /** Shut down the SparkContext. */ def stop() { + postApplicationEnd() ui.stop() - eventLogger.foreach(_.stop()) // Do this only if not stopped already - best case effort. // prevent NPE if stopped more than once. val dagSchedulerCopy = dagScheduler dagScheduler = null if (dagSchedulerCopy != null) { metadataCleaner.cancel() + cleaner.foreach(_.stop()) dagSchedulerCopy.stop() - listenerBus.stop() taskScheduler = null // TODO: Cache.stop()? env.stop() SparkEnv.set(null) ShuffleMapTask.clearCache() ResultTask.clearCache() + listenerBus.stop() + eventLogger.foreach(_.stop()) logInfo("Successfully stopped SparkContext") } else { logInfo("SparkContext already stopped") @@ -915,9 +1035,11 @@ class SparkContext( * Capture the current user callsite and return a formatted version for printing. If the user * has overridden the call site, this will return the user's version. */ - private[spark] def getCallSite(): String = { - val defaultCallSite = Utils.getCallSiteInfo - Option(getLocalProperty("externalCallSite")).getOrElse(defaultCallSite.toString) + private[spark] def getCallSite(): CallSite = { + Option(getLocalProperty("externalCallSite")) match { + case Some(callSite) => CallSite(callSite, long = "") + case None => Utils.getCallSite + } } /** @@ -932,16 +1054,16 @@ class SparkContext( partitions: Seq[Int], allowLocal: Boolean, resultHandler: (Int, U) => Unit) { - partitions.foreach{ p => - require(p >= 0 && p < rdd.partitions.size, s"Invalid partition requested: $p") + if (dagScheduler == null) { + throw new SparkException("SparkContext has been shutdown") } val callSite = getCallSite val cleanedFunc = clean(func) - logInfo("Starting job: " + callSite) + logInfo("Starting job: " + callSite.short) val start = System.nanoTime dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal, resultHandler, localProperties.get) - logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s") + logInfo("Job finished: " + callSite.short + ", took " + (System.nanoTime - start) / 1e9 + " s") rdd.doCheckpoint() } @@ -1012,25 +1134,29 @@ class SparkContext( } /** + * :: DeveloperApi :: * Run a job that can return approximate results. */ + @DeveloperApi def runApproximateJob[T, U, R]( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, evaluator: ApproximateEvaluator[U, R], timeout: Long): PartialResult[R] = { val callSite = getCallSite - logInfo("Starting job: " + callSite) + logInfo("Starting job: " + callSite.short) val start = System.nanoTime val result = dagScheduler.runApproximateJob(rdd, func, evaluator, callSite, timeout, localProperties.get) - logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s") + logInfo("Job finished: " + callSite.short + ", took " + (System.nanoTime - start) / 1e9 + " s") result } /** + * :: Experimental :: * Submit a job for execution and return a FutureJob holding the result. */ + @Experimental def submitJob[T, U, R]( rdd: RDD[T], processPartition: Iterator[T] => U, @@ -1038,9 +1164,6 @@ class SparkContext( resultHandler: (Int, U) => Unit, resultFunc: => R): SimpleFutureAction[R] = { - partitions.foreach{ p => - require(p >= 0 && p < rdd.partitions.size, s"Invalid partition requested: $p") - } val cleanF = clean(processPartition) val callSite = getCallSite val waiter = dagScheduler.submitJob( @@ -1067,6 +1190,16 @@ class SparkContext( dagScheduler.cancelAllJobs() } + /** Cancel a given job if it's scheduled or running */ + private[spark] def cancelJob(jobId: Int) { + dagScheduler.cancelJob(jobId) + } + + /** Cancel a given stage and all jobs associated with it */ + private[spark] def cancelStage(stageId: Int) { + dagScheduler.cancelStage(stageId) + } + /** * Clean a closure to make it ready to serialized and send to tasks * (removes unreferenced variables in $outer's, updates REPL variables) @@ -1095,8 +1228,12 @@ class SparkContext( def defaultParallelism: Int = taskScheduler.defaultParallelism /** Default min number of partitions for Hadoop RDDs when not given by user */ + @deprecated("use defaultMinPartitions", "1.0.0") def defaultMinSplits: Int = math.min(defaultParallelism, 2) + /** Default min number of partitions for Hadoop RDDs when not given by user */ + def defaultMinPartitions: Int = math.min(defaultParallelism, 2) + private val nextShuffleId = new AtomicInteger(0) private[spark] def newShuffleId(): Int = nextShuffleId.getAndIncrement() @@ -1106,6 +1243,16 @@ class SparkContext( /** Register a new RDD, returning its RDD ID */ private[spark] def newRddId(): Int = nextRddId.getAndIncrement() + /** Post the application start event */ + private def postApplicationStart() { + listenerBus.post(SparkListenerApplicationStart(appName, startTime, sparkUser)) + } + + /** Post the application end event */ + private def postApplicationEnd() { + listenerBus.post(SparkListenerApplicationEnd(System.currentTimeMillis)) + } + /** Post the environment update event once the task scheduler is ready */ private def postEnvironmentUpdate() { if (taskScheduler != null) { @@ -1131,10 +1278,14 @@ class SparkContext( */ object SparkContext extends Logging { + private[spark] val SPARK_VERSION = "1.0.0" + 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 SPARK_UNKNOWN_USER = "" implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { @@ -1159,8 +1310,10 @@ object SparkContext extends Logging { // TODO: Add AccumulatorParams for other types, e.g. lists and strings - implicit def rddToPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) = + implicit def rddToPairRDDFunctions[K, V](rdd: RDD[(K, V)]) + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = { new PairRDDFunctions(rdd) + } implicit def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = new AsyncRDDActions(rdd) @@ -1168,7 +1321,7 @@ object SparkContext extends Logging { rdd: RDD[(K, V)]) = new SequenceFileRDDFunctions(rdd) - implicit def rddToOrderedRDDFunctions[K <% Ordered[K]: ClassTag, V: ClassTag]( + implicit def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag]( rdd: RDD[(K, V)]) = new OrderedRDDFunctions[K, V, (K, V)](rdd) @@ -1237,19 +1390,19 @@ object SparkContext extends Logging { * Find the JAR from which a given class was loaded, to make it easy for users to pass * their JARs to SparkContext. */ - def jarOfClass(cls: Class[_]): Seq[String] = { + def jarOfClass(cls: Class[_]): Option[String] = { val uri = cls.getResource("/" + cls.getName.replace('.', '/') + ".class") if (uri != null) { val uriStr = uri.toString if (uriStr.startsWith("jar:file:")) { // URI will be of the form "jar:file:/path/foo.jar!/package/cls.class", // so pull out the /path/foo.jar - List(uriStr.substring("jar:file:".length, uriStr.indexOf('!'))) + Some(uriStr.substring("jar:file:".length, uriStr.indexOf('!'))) } else { - Nil + None } } else { - Nil + None } } @@ -1258,7 +1411,7 @@ object SparkContext extends Logging { * to pass their JARs to SparkContext. In most cases you can call jarOfObject(this) in * your driver program. */ - def jarOfObject(obj: AnyRef): Seq[String] = jarOfClass(obj.getClass) + def jarOfObject(obj: AnyRef): Option[String] = jarOfClass(obj.getClass) /** * Creates a modified version of a SparkConf with the parameters that can be passed separately @@ -1289,8 +1442,8 @@ object SparkContext extends Logging { /** Creates a task scheduler based on a given master URL. Extracted for testing. */ private def createTaskScheduler(sc: SparkContext, master: String): TaskScheduler = { - // Regular expression used for local[N] master format - val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r + // Regular expression used for local[N] and local[*] master formats + val LOCAL_N_REGEX = """local\[([0-9\*]+)\]""".r // Regular expression for local[N, maxRetries], used in tests with failing tasks val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+)\s*,\s*([0-9]+)\]""".r // Regular expression for simulating a Spark cluster of [N, cores, memory] locally @@ -1313,8 +1466,11 @@ object SparkContext extends Logging { scheduler case LOCAL_N_REGEX(threads) => + def localCpuCount = 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 val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) - val backend = new LocalBackend(scheduler, threads.toInt) + val backend = new LocalBackend(scheduler, threadCount) scheduler.initialize(backend) scheduler @@ -1363,8 +1519,8 @@ object SparkContext extends Logging { } catch { // TODO: Enumerate the exact reasons why it can fail // But irrespective of it, it means we cannot proceed ! - case th: Throwable => { - throw new SparkException("YARN mode not available ?", th) + case e: Exception => { + throw new SparkException("YARN mode not available ?", e) } } val backend = new CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) @@ -1379,8 +1535,8 @@ object SparkContext extends Logging { cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl] } catch { - case th: Throwable => { - throw new SparkException("YARN mode not available ?", th) + case e: Exception => { + throw new SparkException("YARN mode not available ?", e) } } @@ -1390,8 +1546,8 @@ object SparkContext extends Logging { val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext]) cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend] } catch { - case th: Throwable => { - throw new SparkException("YARN mode not available ?", th) + case e: Exception => { + throw new SparkException("YARN mode not available ?", e) } } diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 5ceac28fe7afb..8dfa8cc4b5b3f 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -17,6 +17,8 @@ package org.apache.spark +import java.io.File + import scala.collection.JavaConversions._ import scala.collection.mutable import scala.concurrent.Await @@ -25,30 +27,37 @@ import scala.util.Properties import akka.actor._ import com.google.common.collect.MapMaker +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem import org.apache.spark.network.ConnectionManager import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.storage._ import org.apache.spark.util.{AkkaUtils, Utils} /** + * :: DeveloperApi :: * Holds all the runtime environment objects for a running Spark instance (either master or worker), * including the serializer, Akka actor system, block manager, map output tracker, etc. Currently * Spark code finds the SparkEnv through a thread-local variable, so each thread that accesses these * objects needs to have the right SparkEnv set. You can get the current environment with * SparkEnv.get (e.g. after creating a SparkContext) and set it with SparkEnv.set. + * + * NOTE: This is not intended for external use. This is exposed for Shark and may be made private + * in a future release. */ -class SparkEnv private[spark] ( +@DeveloperApi +class SparkEnv ( val executorId: String, val actorSystem: ActorSystem, val serializer: Serializer, val closureSerializer: Serializer, val cacheManager: CacheManager, val mapOutputTracker: MapOutputTracker, - val shuffleFetcher: ShuffleFetcher, + val shuffleManager: ShuffleManager, val broadcastManager: BroadcastManager, val blockManager: BlockManager, val connectionManager: ConnectionManager, @@ -72,7 +81,7 @@ class SparkEnv private[spark] ( pythonWorkers.foreach { case(key, worker) => worker.stop() } httpFileServer.stop() mapOutputTracker.stop() - shuffleFetcher.stop() + shuffleManager.stop() broadcastManager.stop() blockManager.stop() blockManager.master.stop() @@ -91,6 +100,14 @@ class SparkEnv private[spark] ( pythonWorkers.getOrElseUpdate(key, new PythonWorkerFactory(pythonExec, envVars)).create() } } + + private[spark] + def destroyPythonWorker(pythonExec: String, envVars: Map[String, String]) { + synchronized { + val key = (pythonExec, envVars) + pythonWorkers(key).stop() + } + } } object SparkEnv extends Logging { @@ -142,20 +159,25 @@ object SparkEnv extends Logging { conf.set("spark.driver.port", boundPort.toString) } - val classLoader = Thread.currentThread.getContextClassLoader - // Create an instance of the class named by the given Java system property, or by // defaultClassName if the property is not set, and return it as a T def instantiateClass[T](propertyName: String, defaultClassName: String): T = { val name = conf.get(propertyName, defaultClassName) - val cls = Class.forName(name, true, classLoader) - // First try with the constructor that takes SparkConf. If we can't find one, - // use a no-arg constructor instead. + val cls = Class.forName(name, true, Utils.getContextOrSparkClassLoader) + // Look for a constructor taking a SparkConf and a boolean isDriver, then one taking just + // SparkConf, then one taking no arguments try { - cls.getConstructor(classOf[SparkConf]).newInstance(conf).asInstanceOf[T] + cls.getConstructor(classOf[SparkConf], java.lang.Boolean.TYPE) + .newInstance(conf, new java.lang.Boolean(isDriver)) + .asInstanceOf[T] } catch { case _: NoSuchMethodException => - cls.getConstructor().newInstance().asInstanceOf[T] + try { + cls.getConstructor(classOf[SparkConf]).newInstance(conf).asInstanceOf[T] + } catch { + case _: NoSuchMethodException => + cls.getConstructor().newInstance().asInstanceOf[T] + } } } @@ -180,12 +202,24 @@ object SparkEnv extends Logging { } } + val mapOutputTracker = if (isDriver) { + new MapOutputTrackerMaster(conf) + } else { + new MapOutputTrackerWorker(conf) + } + + // Have to assign trackerActor after initialization as MapOutputTrackerActor + // requires the MapOutputTracker itself + mapOutputTracker.trackerActor = registerOrLookup( + "MapOutputTracker", + new MapOutputTrackerMasterActor(mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], conf)) + val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf) val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, - serializer, conf, securityManager) + serializer, conf, securityManager, mapOutputTracker) val connectionManager = blockManager.connectionManager @@ -193,20 +227,6 @@ object SparkEnv extends Logging { val cacheManager = new CacheManager(blockManager) - // Have to assign trackerActor after initialization as MapOutputTrackerActor - // requires the MapOutputTracker itself - val mapOutputTracker = if (isDriver) { - new MapOutputTrackerMaster(conf) - } else { - new MapOutputTracker(conf) - } - mapOutputTracker.trackerActor = registerOrLookup( - "MapOutputTracker", - new MapOutputTrackerMasterActor(mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], conf)) - - val shuffleFetcher = instantiateClass[ShuffleFetcher]( - "spark.shuffle.fetcher", "org.apache.spark.BlockStoreShuffleFetcher") - val httpFileServer = new HttpFileServer(securityManager) httpFileServer.initialize() conf.set("spark.fileserver.uri", httpFileServer.serverUri) @@ -227,6 +247,9 @@ object SparkEnv extends Logging { "." } + val shuffleManager = instantiateClass[ShuffleManager]( + "spark.shuffle.manager", "org.apache.spark.shuffle.hash.HashShuffleManager") + // 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 " + @@ -240,7 +263,7 @@ object SparkEnv extends Logging { closureSerializer, cacheManager, mapOutputTracker, - shuffleFetcher, + shuffleManager, broadcastManager, blockManager, connectionManager, @@ -263,11 +286,11 @@ object SparkEnv extends Logging { addedJars: Seq[String], addedFiles: Seq[String]): Map[String, Seq[(String, String)]] = { + import Properties._ val jvmInformation = Seq( - ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), - ("Java Home", Properties.javaHome), - ("Scala Version", Properties.versionString), - ("Scala Home", Properties.scalaHome) + ("Java Version", s"$javaVersion ($javaVendor)"), + ("Java Home", javaHome), + ("Scala Version", versionString) ).sorted // Spark properties @@ -282,18 +305,15 @@ object SparkEnv extends Logging { // System properties that are not java classpaths val systemProperties = System.getProperties.iterator.toSeq - val otherProperties = systemProperties.filter { case (k, v) => + val otherProperties = systemProperties.filter { case (k, _) => k != "java.class.path" && !k.startsWith("spark.") }.sorted // Class paths including all added jars and files - val classPathProperty = systemProperties.find { case (k, v) => - k == "java.class.path" - }.getOrElse(("", "")) - val classPathEntries = classPathProperty._2 - .split(conf.get("path.separator", ":")) - .filterNot(e => e.isEmpty) - .map(e => (e, "System Classpath")) + val classPathEntries = javaClassPath + .split(File.pathSeparator) + .filterNot(_.isEmpty) + .map((_, "System Classpath")) val addedJarsAndFiles = (addedJars ++ addedFiles).map((_, "Added By User")) val classPaths = (addedJarsAndFiles ++ classPathEntries).sorted diff --git a/core/src/main/scala/org/apache/spark/SparkException.scala b/core/src/main/scala/org/apache/spark/SparkException.scala index d34e47e8cac22..4351ed74b67fc 100644 --- a/core/src/main/scala/org/apache/spark/SparkException.scala +++ b/core/src/main/scala/org/apache/spark/SparkException.scala @@ -20,5 +20,5 @@ package org.apache.spark class SparkException(message: String, cause: Throwable) extends Exception(message, cause) { - def this(message: String) = this(message, null) + def this(message: String) = this(message, null) } diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index b92ea01a877f7..f6703986bdf11 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -42,7 +42,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) private val now = new Date() private val conf = new SerializableWritable(jobConf) - + private var jobID = 0 private var splitID = 0 private var attemptID = 0 @@ -58,8 +58,8 @@ class SparkHadoopWriter(@transient jobConf: JobConf) def preSetup() { setIDs(0, 0, 0) HadoopRDD.addLocalConfiguration("", 0, 0, 0, conf.value) - - val jCtxt = getJobContext() + + val jCtxt = getJobContext() getOutputCommitter().setupJob(jCtxt) } @@ -74,7 +74,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) val numfmt = NumberFormat.getInstance() numfmt.setMinimumIntegerDigits(5) numfmt.setGroupingUsed(false) - + val outputName = "part-" + numfmt.format(splitID) val path = FileOutputFormat.getOutputPath(conf.value) val fs: FileSystem = { @@ -85,7 +85,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) } } - getOutputCommitter().setupTask(getTaskContext()) + getOutputCommitter().setupTask(getTaskContext()) writer = getOutputFormat().getRecordWriter(fs, conf.value, outputName, Reporter.NULL) } @@ -103,18 +103,18 @@ class SparkHadoopWriter(@transient jobConf: JobConf) def commit() { val taCtxt = getTaskContext() - val cmtr = getOutputCommitter() + val cmtr = getOutputCommitter() if (cmtr.needsTaskCommit(taCtxt)) { try { cmtr.commitTask(taCtxt) logInfo (taID + ": Committed") } catch { - case e: IOException => { + case e: IOException => { logError("Error committing the output of task: " + taID.value, e) cmtr.abortTask(taCtxt) throw e } - } + } } else { logWarning ("No need to commit output of task: " + taID.value) } @@ -144,7 +144,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) } private def getJobContext(): JobContext = { - if (jobContext == null) { + if (jobContext == null) { jobContext = newJobContext(conf.value, jID.value) } jobContext @@ -175,7 +175,7 @@ object SparkHadoopWriter { val jobtrackerID = formatter.format(time) new JobID(jobtrackerID, id) } - + def createPathFromString(path: String, conf: JobConf): Path = { if (path == null) { throw new IllegalArgumentException("Output path is null") diff --git a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala index a2a871cbd3c31..65003b6ac6a0a 100644 --- a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala +++ b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala @@ -44,12 +44,12 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg * configurable in the future. */ private var saslClient: SaslClient = Sasl.createSaslClient(Array[String](SparkSaslServer.DIGEST), - null, null, SparkSaslServer.SASL_DEFAULT_REALM, SparkSaslServer.SASL_PROPS, + null, null, SparkSaslServer.SASL_DEFAULT_REALM, SparkSaslServer.SASL_PROPS, new SparkSaslClientCallbackHandler(securityMgr)) /** * Used to initiate SASL handshake with server. - * @return response to challenge if needed + * @return response to challenge if needed */ def firstToken(): Array[Byte] = { synchronized { @@ -86,7 +86,7 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg } /** - * Disposes of any system resources or security-sensitive information the + * Disposes of any system resources or security-sensitive information the * SaslClient might be using. */ def dispose() { @@ -110,11 +110,11 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg private class SparkSaslClientCallbackHandler(securityMgr: SecurityManager) extends CallbackHandler { - private val userName: String = - SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes()) + private val userName: String = + SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes("utf-8")) private val secretKey = securityMgr.getSecretKey() - private val userPassword: Array[Char] = - SparkSaslServer.encodePassword(if (secretKey != null) secretKey.getBytes() else "".getBytes()) + private val userPassword: Array[Char] = SparkSaslServer.encodePassword( + if (secretKey != null) secretKey.getBytes("utf-8") else "".getBytes("utf-8")) /** * Implementation used to respond to SASL request from the server. @@ -138,7 +138,7 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg rc.setText(rc.getDefaultText()) } case cb: RealmChoiceCallback => {} - case cb: Callback => throw + case cb: Callback => throw new UnsupportedCallbackException(cb, "handle: Unrecognized SASL client callback") } } diff --git a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala index 11fcb2ae3a5c5..f6b0a9132aca4 100644 --- a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala +++ b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala @@ -64,7 +64,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi } /** - * Disposes of any system resources or security-sensitive information the + * Disposes of any system resources or security-sensitive information the * SaslServer might be using. */ def dispose() { @@ -88,8 +88,8 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi private class SparkSaslDigestCallbackHandler(securityMgr: SecurityManager) extends CallbackHandler { - private val userName: String = - SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes()) + private val userName: String = + SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes("utf-8")) override def handle(callbacks: Array[Callback]) { logDebug("In the sasl server callback handler") @@ -101,7 +101,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi case pc: PasswordCallback => { logDebug("handle: SASL server callback: setting userPassword") val password: Array[Char] = - SparkSaslServer.encodePassword(securityMgr.getSecretKey().getBytes()) + SparkSaslServer.encodePassword(securityMgr.getSecretKey().getBytes("utf-8")) pc.setPassword(password) } case rc: RealmCallback => { @@ -123,7 +123,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi ac.setAuthorizedID(authzid) } } - case cb: Callback => throw + case cb: Callback => throw new UnsupportedCallbackException(cb, "handle: Unrecognized SASL DIGEST-MD5 Callback") } } @@ -159,7 +159,7 @@ private[spark] object SparkSaslServer { * @return Base64-encoded string */ def encodeIdentifier(identifier: Array[Byte]): String = { - new String(Base64.encodeBase64(identifier)) + new String(Base64.encodeBase64(identifier), "utf-8") } /** @@ -168,7 +168,7 @@ private[spark] object SparkSaslServer { * @return password as a char array. */ def encodePassword(password: Array[Byte]): Array[Char] = { - new String(Base64.encodeBase64(password)).toCharArray() + new String(Base64.encodeBase64(password), "utf-8").toCharArray() } } diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index be53ca2968cfb..51f40c339d13c 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -19,16 +19,21 @@ package org.apache.spark import scala.collection.mutable.ArrayBuffer +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics +/** + * :: DeveloperApi :: + * Contextual information about a task which can be read or mutated during execution. + */ +@DeveloperApi class TaskContext( - val stageId: Int, - val partitionId: Int, - val attemptId: Long, - val runningLocally: Boolean = false, - @volatile var interrupted: Boolean = false, - private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty() -) extends Serializable { + val stageId: Int, + val partitionId: Int, + val attemptId: Long, + val runningLocally: Boolean = false, + private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty) + extends Serializable { @deprecated("use partitionId", "0.8.1") def splitId = partitionId @@ -36,9 +41,16 @@ class TaskContext( // List of callback functions to execute when the task completes. @transient private val onCompleteCallbacks = new ArrayBuffer[() => Unit] + // Whether the corresponding task has been killed. + @volatile var interrupted: Boolean = false + + // Whether the task has completed, before the onCompleteCallbacks are executed. + @volatile var completed: Boolean = false + /** * Add a callback function to be executed on task completion. An example use * is for HadoopRDD to register a callback to close the input stream. + * Will be called in any situation - success, failure, or cancellation. * @param f Callback function. */ def addOnCompleteCallback(f: () => Unit) { @@ -46,7 +58,8 @@ class TaskContext( } def executeOnCompleteCallbacks() { + completed = true // Process complete callbacks in the reverse order of registration - onCompleteCallbacks.reverse.foreach{_()} + onCompleteCallbacks.reverse.foreach { _() } } } diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index f1a753b6ab8a9..a3074916d13e7 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -17,29 +17,35 @@ package org.apache.spark +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId /** + * :: DeveloperApi :: * Various possible reasons why a task ended. The low-level TaskScheduler is supposed to retry * tasks several times for "ephemeral" failures, and only report back failures that require some * old stages to be resubmitted, such as shuffle map fetch failures. */ -private[spark] sealed trait TaskEndReason +@DeveloperApi +sealed trait TaskEndReason -private[spark] case object Success extends TaskEndReason +@DeveloperApi +case object Success extends TaskEndReason -private[spark] +@DeveloperApi case object Resubmitted extends TaskEndReason // Task was finished earlier but we've now lost it -private[spark] case class FetchFailed( +@DeveloperApi +case class FetchFailed( bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, reduceId: Int) extends TaskEndReason -private[spark] case class ExceptionFailure( +@DeveloperApi +case class ExceptionFailure( className: String, description: String, stackTrace: Array[StackTraceElement], @@ -47,21 +53,28 @@ private[spark] case class ExceptionFailure( extends TaskEndReason /** + * :: DeveloperApi :: * The task finished successfully, but the result was lost from the executor's block manager before * it was fetched. */ -private[spark] case object TaskResultLost extends TaskEndReason +@DeveloperApi +case object TaskResultLost extends TaskEndReason -private[spark] case object TaskKilled extends TaskEndReason +@DeveloperApi +case object TaskKilled extends TaskEndReason /** + * :: DeveloperApi :: * The task failed because the executor that it was running on was lost. This may happen because * the task crashed the JVM. */ -private[spark] case object ExecutorLostFailure extends TaskEndReason +@DeveloperApi +case object ExecutorLostFailure extends TaskEndReason /** + * :: DeveloperApi :: * We don't know why the task ended -- for example, because of a ClassNotFound exception when * deserializing the task result. */ -private[spark] case object UnknownReason extends TaskEndReason +@DeveloperApi +case object UnknownReason extends TaskEndReason diff --git a/core/src/main/scala/org/apache/spark/TaskKilledException.scala b/core/src/main/scala/org/apache/spark/TaskKilledException.scala new file mode 100644 index 0000000000000..ad487c4efb87a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/TaskKilledException.scala @@ -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. + */ + +package org.apache.spark + +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * Exception thrown when a task is explicitly killed (i.e., task failure is expected). + */ +@DeveloperApi +class TaskKilledException extends RuntimeException diff --git a/core/src/test/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala similarity index 77% rename from core/src/test/scala/org/apache/spark/TestUtils.scala rename to core/src/main/scala/org/apache/spark/TestUtils.scala index 1611d09652d40..885c6829a2d72 100644 --- a/core/src/test/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -26,7 +26,14 @@ import scala.collection.JavaConversions._ import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider} import com.google.common.io.Files -object TestUtils { +/** + * Utilities for tests. Included in main codebase since it's used by multiple + * projects. + * + * TODO: See if we can move this to the test codebase by specifying + * test dependencies between projects. + */ +private[spark] object TestUtils { /** * Create a jar that defines classes with the given names. @@ -34,13 +41,15 @@ object TestUtils { * Note: if this is used during class loader tests, class names should be unique * in order to avoid interference between tests. */ - def createJarWithClasses(classNames: Seq[String]): URL = { + def createJarWithClasses(classNames: Seq[String], value: String = ""): URL = { val tempDir = Files.createTempDir() - val files = for (name <- classNames) yield createCompiledClass(name, tempDir) + tempDir.deleteOnExit() + val files = for (name <- classNames) yield createCompiledClass(name, tempDir, value) val jarFile = new File(tempDir, "testJar-%s.jar".format(System.currentTimeMillis())) createJar(files, jarFile) } + /** * Create a jar file that contains this set of files. All files will be located at the root * of the jar. @@ -80,9 +89,11 @@ object TestUtils { } /** Creates a compiled class with the given name. Class file will be placed in destDir. */ - def createCompiledClass(className: String, destDir: File): File = { + def createCompiledClass(className: String, destDir: File, value: String = ""): File = { val compiler = ToolProvider.getSystemJavaCompiler - val sourceFile = new JavaSourceFromString(className, s"public class $className {}") + val sourceFile = new JavaSourceFromString(className, + "public class " + className + " { @Override public String toString() { " + + "return \"" + value + "\";}}") // Calling this outputs a class file in pwd. It's easier to just rename the file than // build a custom FileManager that controls the output location. @@ -90,9 +101,14 @@ object TestUtils { val fileName = className + ".class" val result = new File(fileName) - if (!result.exists()) throw new Exception("Compiled file not found: " + fileName) + assert(result.exists(), "Compiled file not found: " + result.getAbsolutePath()) val out = new File(destDir, fileName) - result.renameTo(out) + + // renameTo cannot handle in and out files in different filesystems + // use google's Files.move instead + Files.move(result, out) + + assert(out.exists(), "Destination file not moved: " + out.getAbsolutePath()) out } } diff --git a/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java b/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java new file mode 100644 index 0000000000000..db7b25c727d34 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.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.annotation; + +import java.lang.annotation.*; + +/** + * A new component of Spark which may have unstable API's. + * + * NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first + * line of the comment must be ":: AlphaComponent ::" 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 AlphaComponent {} diff --git a/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java b/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java new file mode 100644 index 0000000000000..0ecef6db0e039 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.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.annotation; + +import java.lang.annotation.*; + +/** + * A lower-level, unstable API intended for developers. + * + * Developer API's might change or be removed in minor versions of Spark. + * + * NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first + * line of the comment must be ":: DeveloperApi ::" 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 DeveloperApi {} diff --git a/core/src/main/scala/org/apache/spark/annotation/Experimental.java b/core/src/main/scala/org/apache/spark/annotation/Experimental.java new file mode 100644 index 0000000000000..ff8120291455f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotation/Experimental.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.annotation; + +import java.lang.annotation.*; + +/** + * An experimental user-facing API. + * + * Experimental API's might change or be removed in minor versions of Spark, or be adopted as + * first-class Spark API's. + * + * NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first + * line of the comment must be ":: Experimental ::" 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 Experimental {} diff --git a/core/src/main/scala/org/apache/spark/annotation/package-info.java b/core/src/main/scala/org/apache/spark/annotation/package-info.java new file mode 100644 index 0000000000000..12c7afe6f108a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotation/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Spark annotations to mark an API experimental or intended only for advanced usages by developers. + * This package consist of these annotations, which are used project wide and are reflected in + * Scala and Java docs. + */ +package org.apache.spark.annotation; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/annotation/package.scala b/core/src/main/scala/org/apache/spark/annotation/package.scala new file mode 100644 index 0000000000000..c3f4026a29d44 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotation/package.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Spark annotations to mark an API experimental or intended only for advanced usages by developers. + * This package consist of these annotations, which are used project wide and are reflected in + * Scala and Java docs. + */ +package object annotation 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 f816bb43a5b44..a6123bd108c11 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 @@ -19,15 +19,18 @@ package org.apache.spark.api.java import java.lang.{Double => JDouble} +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.Partitioner import org.apache.spark.SparkContext.doubleRDDToDoubleRDDFunctions +import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.util.StatCounter +import org.apache.spark.util.Utils class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, JavaDoubleRDD] { @@ -131,7 +134,13 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, Ja /** * Return a sampled subset of this RDD. */ - def sample(withReplacement: Boolean, fraction: JDouble, seed: Int): JavaDoubleRDD = + def sample(withReplacement: Boolean, fraction: JDouble): JavaDoubleRDD = + sample(withReplacement, fraction, Utils.random.nextLong) + + /** + * Return a sampled subset of this RDD. + */ + def sample(withReplacement: Boolean, fraction: JDouble, seed: Long): JavaDoubleRDD = fromRDD(srdd.sample(withReplacement, fraction, seed)) /** @@ -184,14 +193,26 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, Ja def meanApprox(timeout: Long, confidence: JDouble): PartialResult[BoundedDouble] = srdd.meanApprox(timeout, confidence) - /** (Experimental) Approximate operation to return the mean within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the mean within a timeout. + */ + @Experimental def meanApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.meanApprox(timeout) - /** (Experimental) Approximate operation to return the sum within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the sum within a timeout. + */ + @Experimental def sumApprox(timeout: Long, confidence: JDouble): PartialResult[BoundedDouble] = srdd.sumApprox(timeout, confidence) - /** (Experimental) Approximate operation to return the sum within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the sum within a timeout. + */ + @Experimental def sumApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.sumApprox(timeout) /** 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 9596dbaf75488..4f3081433a542 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 @@ -18,24 +18,28 @@ package org.apache.spark.api.java import java.util.{Comparator, List => JList} +import java.lang.{Iterable => JIterable} import scala.collection.JavaConversions._ +import scala.language.implicitConversions import scala.reflect.ClassTag import com.google.common.base.Optional import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{JobConf, OutputFormat} -import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, Job} +import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.spark.{HashPartitioner, Partitioner} import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext.rddToPairRDDFunctions +import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, PairFunction} import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.{OrderedRDDFunctions, RDD} import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) (implicit val kClassTag: ClassTag[K], implicit val vClassTag: ClassTag[V]) @@ -116,7 +120,13 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * Return a sampled subset of this RDD. */ - def sample(withReplacement: Boolean, fraction: Double, seed: Int): JavaPairRDD[K, V] = + def sample(withReplacement: Boolean, fraction: Double): JavaPairRDD[K, V] = + sample(withReplacement, fraction, Utils.random.nextLong) + + /** + * Return a sampled subset of this RDD. + */ + def sample(withReplacement: Boolean, fraction: Double, seed: Long): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.sample(withReplacement, fraction, seed)) /** @@ -200,20 +210,68 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) def countByKey(): java.util.Map[K, Long] = mapAsJavaMap(rdd.countByKey()) /** - * (Experimental) Approximate version of countByKey that can return a partial result if it does + * :: Experimental :: + * Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. */ + @Experimental def countByKeyApprox(timeout: Long): PartialResult[java.util.Map[K, BoundedDouble]] = rdd.countByKeyApprox(timeout).map(mapAsJavaMap) /** - * (Experimental) Approximate version of countByKey that can return a partial result if it does + * :: Experimental :: + * Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. */ + @Experimental def countByKeyApprox(timeout: Long, confidence: Double = 0.95) : PartialResult[java.util.Map[K, BoundedDouble]] = rdd.countByKeyApprox(timeout, confidence).map(mapAsJavaMap) + /** + * Aggregate the values of each key, 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, + * as in scala.TraversableOnce. 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. + */ + def aggregateByKey[U](zeroValue: U, partitioner: Partitioner, seqFunc: JFunction2[U, V, U], + combFunc: JFunction2[U, U, U]): JavaPairRDD[K, U] = { + implicit val ctag: ClassTag[U] = fakeClassTag + fromRDD(rdd.aggregateByKey(zeroValue, partitioner)(seqFunc, combFunc)) + } + + /** + * Aggregate the values of each key, 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, + * as in scala.TraversableOnce. 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. + */ + def aggregateByKey[U](zeroValue: U, numPartitions: Int, seqFunc: JFunction2[U, V, U], + combFunc: JFunction2[U, U, U]): JavaPairRDD[K, U] = { + implicit val ctag: ClassTag[U] = fakeClassTag + fromRDD(rdd.aggregateByKey(zeroValue, numPartitions)(seqFunc, combFunc)) + } + + /** + * Aggregate the values of each key, 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. + */ + def aggregateByKey[U](zeroValue: U, seqFunc: JFunction2[U, V, U], combFunc: JFunction2[U, U, U]): + JavaPairRDD[K, U] = { + implicit val ctag: ClassTag[U] = fakeClassTag + fromRDD(rdd.aggregateByKey(zeroValue)(seqFunc, combFunc)) + } + /** * Merge the values for each key using an associative function and a neutral "zero value" which * may be added to the result an arbitrary number of times, and must not change the result @@ -249,15 +307,23 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * Group the values for each key in the RDD into a single sequence. Allows controlling the * partitioning of the resulting key-value pair RDD by passing a Partitioner. + * + * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over + * each key, using [[JavaPairRDD.reduceByKey]] or [[JavaPairRDD.combineByKey]] + * will provide much better performance. */ - def groupByKey(partitioner: Partitioner): JavaPairRDD[K, JList[V]] = + def groupByKey(partitioner: Partitioner): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey(partitioner))) /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with into `numPartitions` partitions. + * + * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over + * each key, using [[JavaPairRDD.reduceByKey]] or [[JavaPairRDD.combineByKey]] + * will provide much better performance. */ - def groupByKey(numPartitions: Int): JavaPairRDD[K, JList[V]] = + def groupByKey(numPartitions: Int): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey(numPartitions))) /** @@ -366,8 +432,12 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with the existing partitioner/parallelism level. + * + * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over + * each key, using [[JavaPairRDD.reduceByKey]] or [[JavaPairRDD.combineByKey]] + * will provide much better performance. */ - def groupByKey(): JavaPairRDD[K, JList[V]] = + def groupByKey(): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey())) /** @@ -462,7 +532,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * list of values for that key in `this` as well as `other`. */ def cogroup[W](other: JavaPairRDD[K, W], partitioner: Partitioner) - : JavaPairRDD[K, (JList[V], JList[W])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W])] = fromRDD(cogroupResultToJava(rdd.cogroup(other, partitioner))) /** @@ -470,14 +540,26 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], - partitioner: Partitioner): JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + partitioner: Partitioner): JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, partitioner))) + /** + * For each key k in `this` or `other1` or `other2` or `other3`, + * return a resulting RDD that contains a tuple with the list of values + * for that key in `this`, `other1`, `other2` and `other3`. + */ + def cogroup[W1, W2, W3](other1: JavaPairRDD[K, W1], + other2: JavaPairRDD[K, W2], + other3: JavaPairRDD[K, W3], + partitioner: Partitioner) + : JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2], JIterable[W3])] = + fromRDD(cogroupResult3ToJava(rdd.cogroup(other1, other2, other3, partitioner))) + /** * 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: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = + def cogroup[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JIterable[V], JIterable[W])] = fromRDD(cogroupResultToJava(rdd.cogroup(other))) /** @@ -485,15 +567,26 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2]) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2))) + /** + * For each key k in `this` or `other1` or `other2` or `other3`, + * return a resulting RDD that contains a tuple with the list of values + * for that key in `this`, `other1`, `other2` and `other3`. + */ + def cogroup[W1, W2, W3](other1: JavaPairRDD[K, W1], + other2: JavaPairRDD[K, W2], + other3: JavaPairRDD[K, W3]) + : JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2], JIterable[W3])] = + fromRDD(cogroupResult3ToJava(rdd.cogroup(other1, other2, other3))) + /** * 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: JavaPairRDD[K, W], numPartitions: Int) - : JavaPairRDD[K, (JList[V], JList[W])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W])] = fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) /** @@ -501,18 +594,37 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], numPartitions: Int) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, numPartitions))) + /** + * For each key k in `this` or `other1` or `other2` or `other3`, + * return a resulting RDD that contains a tuple with the list of values + * for that key in `this`, `other1`, `other2` and `other3`. + */ + def cogroup[W1, W2, W3](other1: JavaPairRDD[K, W1], + other2: JavaPairRDD[K, W2], + other3: JavaPairRDD[K, W3], + numPartitions: Int) + : JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2], JIterable[W3])] = + fromRDD(cogroupResult3ToJava(rdd.cogroup(other1, other2, other3, numPartitions))) + /** Alias for cogroup. */ - def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = + def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JIterable[V], JIterable[W])] = fromRDD(cogroupResultToJava(rdd.groupWith(other))) /** Alias for cogroup. */ def groupWith[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2]) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2])] = fromRDD(cogroupResult2ToJava(rdd.groupWith(other1, other2))) + /** Alias for cogroup. */ + def groupWith[W1, W2, W3](other1: JavaPairRDD[K, W1], + other2: JavaPairRDD[K, W2], + other3: JavaPairRDD[K, W3]) + : JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2], JIterable[W3])] = + fromRDD(cogroupResult3ToJava(rdd.groupWith(other1, other2, other3))) + /** * 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. @@ -619,10 +731,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * order of the keys). */ def sortByKey(comp: Comparator[K], ascending: Boolean): JavaPairRDD[K, V] = { - class KeyOrdering(val a: K) extends Ordered[K] { - override def compare(b: K) = comp.compare(a, b) - } - implicit def toOrdered(x: K): Ordered[K] = new KeyOrdering(x) + implicit val ordering = comp // Allow implicit conversion of Comparator to Ordering. fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending)) } @@ -633,10 +742,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * order of the keys). */ def sortByKey(comp: Comparator[K], ascending: Boolean, numPartitions: Int): JavaPairRDD[K, V] = { - class KeyOrdering(val a: K) extends Ordered[K] { - override def compare(b: K) = comp.compare(a, b) - } - implicit def toOrdered(x: K): Ordered[K] = new KeyOrdering(x) + implicit val ordering = comp // Allow implicit conversion of Comparator to Ordering. fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending, numPartitions)) } @@ -652,38 +758,47 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * Return approximate number of distinct values for each key in this RDD. - * The accuracy of approximation can be controlled through the relative standard deviation - * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vise versa. Uses the provided - * Partitioner to partition the output RDD. + * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. + * @param partitioner partitioner of the resulting RDD. */ - def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): JavaRDD[(K, Long)] = { - rdd.countApproxDistinctByKey(relativeSD, partitioner) + def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): JavaPairRDD[K, Long] = + { + fromRDD(rdd.countApproxDistinctByKey(relativeSD, partitioner)) } /** - * Return approximate number of distinct values for each key this RDD. - * The accuracy of approximation can be controlled through the relative standard deviation - * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vise versa. The default value of - * relativeSD is 0.05. Hash-partitions the output RDD using the existing partitioner/parallelism - * level. + * Return approximate number of distinct values for each key in this RDD. + * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. + * @param numPartitions number of partitions of the resulting RDD. */ - def countApproxDistinctByKey(relativeSD: Double = 0.05): JavaRDD[(K, Long)] = { - rdd.countApproxDistinctByKey(relativeSD) + def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): JavaPairRDD[K, Long] = { + fromRDD(rdd.countApproxDistinctByKey(relativeSD, numPartitions)) } - /** * Return approximate number of distinct values for each key in this RDD. - * The accuracy of approximation can be controlled through the relative standard deviation - * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vise versa. HashPartitions the - * output RDD into numPartitions. * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. */ - def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): JavaRDD[(K, Long)] = { - rdd.countApproxDistinctByKey(relativeSD, numPartitions) + def countApproxDistinctByKey(relativeSD: Double): JavaPairRDD[K, Long] = { + fromRDD(rdd.countApproxDistinctByKey(relativeSD)) } /** Assign a name to this RDD */ @@ -695,21 +810,31 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) object JavaPairRDD { private[spark] - def groupByResultToJava[K: ClassTag, T](rdd: RDD[(K, Seq[T])]): RDD[(K, JList[T])] = { - rddToPairRDDFunctions(rdd).mapValues(seqAsJavaList) + def groupByResultToJava[K: ClassTag, T](rdd: RDD[(K, Iterable[T])]): RDD[(K, JIterable[T])] = { + rddToPairRDDFunctions(rdd).mapValues(asJavaIterable) } private[spark] def cogroupResultToJava[K: ClassTag, V, W]( - rdd: RDD[(K, (Seq[V], Seq[W]))]): RDD[(K, (JList[V], JList[W]))] = { - rddToPairRDDFunctions(rdd).mapValues(x => (seqAsJavaList(x._1), seqAsJavaList(x._2))) + rdd: RDD[(K, (Iterable[V], Iterable[W]))]): RDD[(K, (JIterable[V], JIterable[W]))] = { + rddToPairRDDFunctions(rdd).mapValues(x => (asJavaIterable(x._1), asJavaIterable(x._2))) } private[spark] def cogroupResult2ToJava[K: ClassTag, V, W1, W2]( - rdd: RDD[(K, (Seq[V], Seq[W1], Seq[W2]))]): RDD[(K, (JList[V], JList[W1], JList[W2]))] = { + rdd: RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))]) + : RDD[(K, (JIterable[V], JIterable[W1], JIterable[W2]))] = { + rddToPairRDDFunctions(rdd) + .mapValues(x => (asJavaIterable(x._1), asJavaIterable(x._2), asJavaIterable(x._3))) + } + + private[spark] + def cogroupResult3ToJava[K: ClassTag, V, W1, W2, W3]( + rdd: RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))]) + : RDD[(K, (JIterable[V], JIterable[W1], JIterable[W2], JIterable[W3]))] = { rddToPairRDDFunctions(rdd) - .mapValues(x => (seqAsJavaList(x._1), seqAsJavaList(x._2), seqAsJavaList(x._3))) + .mapValues(x => + (asJavaIterable(x._1), asJavaIterable(x._2), asJavaIterable(x._3), asJavaIterable(x._4))) } def fromRDD[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]): JavaPairRDD[K, V] = { 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 01d9357a2556d..86fb374bef1e3 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 @@ -17,12 +17,17 @@ package org.apache.spark.api.java +import java.util.Comparator + +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark._ +import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) extends JavaRDDLike[T, JavaRDD[T]] { @@ -97,9 +102,37 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) /** * Return a sampled subset of this RDD. */ - def sample(withReplacement: Boolean, fraction: Double, seed: Int): JavaRDD[T] = + def sample(withReplacement: Boolean, fraction: Double): JavaRDD[T] = + sample(withReplacement, fraction, Utils.random.nextLong) + + /** + * Return a sampled subset of this RDD. + */ + def sample(withReplacement: Boolean, fraction: Double, seed: Long): JavaRDD[T] = wrapRDD(rdd.sample(withReplacement, fraction, seed)) + + /** + * Randomly splits this RDD with the provided weights. + * + * @param weights weights for splits, will be normalized if they don't sum to 1 + * + * @return split RDDs in an array + */ + def randomSplit(weights: Array[Double]): Array[JavaRDD[T]] = + randomSplit(weights, Utils.random.nextLong) + + /** + * Randomly splits this RDD with the provided weights. + * + * @param weights weights for splits, will be normalized if they don't sum to 1 + * @param seed random seed + * + * @return split RDDs in an array + */ + def randomSplit(weights: Array[Double], seed: Long): Array[JavaRDD[T]] = + rdd.randomSplit(weights, seed).map(wrapRDD) + /** * Return the union of this RDD and another one. Any identical elements will appear multiple * times (use `.distinct()` to eliminate them). @@ -142,6 +175,19 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) rdd.setName(name) this } + + /** + * Return this RDD sorted by the given key function. + */ + def sortBy[S](f: JFunction[T, S], ascending: Boolean, numPartitions: Int): JavaRDD[T] = { + import scala.collection.JavaConverters._ + def fn = (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 + wrapRDD(rdd.sortBy(fn, ascending, numPartitions)) + } + } object JavaRDD { 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 e03b8e78d5f52..f917cfd1419ec 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,7 +17,8 @@ package org.apache.spark.api.java -import java.util.{Comparator, List => JList} +import java.util.{Comparator, List => JList, Iterator => JIterator} +import java.lang.{Iterable => JIterable, Long => JLong} import scala.collection.JavaConversions._ import scala.reflect.ClassTag @@ -26,12 +27,14 @@ import com.google.common.base.Optional import org.apache.hadoop.io.compress.CompressionCodec import org.apache.spark.{Partition, SparkContext, TaskContext} +import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaPairRDD._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, _} import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def wrapRDD(rdd: RDD[T]): This @@ -40,8 +43,11 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def rdd: RDD[T] - /** Set of partitions in this RDD. */ + @deprecated("Use partitions() instead.", "1.1.0") def splits: JList[Partition] = new java.util.ArrayList(rdd.partitions.toSeq) + + /** Set of partitions in this RDD. */ + def partitions: JList[Partition] = new java.util.ArrayList(rdd.partitions.toSeq) /** The [[org.apache.spark.SparkContext]] that this RDD was created on. */ def context: SparkContext = rdd.context @@ -72,11 +78,11 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return a new RDD by applying a function to each partition of this RDD, while tracking the index * of the original partition. */ - def mapPartitionsWithIndex[R: ClassTag]( + def mapPartitionsWithIndex[R]( f: JFunction2[java.lang.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)) + preservesPartitioning)(fakeClassTag))(fakeClassTag) /** * Return a new RDD by applying a function to all elements of this RDD. @@ -203,7 +209,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JList[T]] = { + def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JIterable[T]] = { implicit val ctagK: ClassTag[K] = fakeClassTag implicit val ctagV: ClassTag[JList[T]] = fakeClassTag JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f)(fakeClassTag))) @@ -213,7 +219,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JList[T]] = { + def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JIterable[T]] = { implicit val ctagK: ClassTag[K] = fakeClassTag implicit val ctagV: ClassTag[JList[T]] = fakeClassTag JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(fakeClassTag[K]))) @@ -261,6 +267,26 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { rdd.zipPartitions(other.rdd)(fn)(other.classTag, fakeClassTag[V]))(fakeClassTag[V]) } + /** + * Zips this 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 [[org.apache.spark.rdd.RDD#zipWithIndex]]. + */ + def zipWithUniqueId(): JavaPairRDD[T, JLong] = { + JavaPairRDD.fromRDD(rdd.zipWithUniqueId()).asInstanceOf[JavaPairRDD[T, JLong]] + } + + /** + * Zips this 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 is similar to Scala's zipWithIndex but it uses Long instead of Int as the index type. + * This method needs to trigger a spark job when this RDD contains more than one partitions. + */ + def zipWithIndex(): JavaPairRDD[T, JLong] = { + JavaPairRDD.fromRDD(rdd.zipWithIndex()).asInstanceOf[JavaPairRDD[T, JLong]] + } + // Actions (launch a job to return a value to the user program) /** @@ -280,6 +306,17 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { new java.util.ArrayList(arr) } + /** + * Return an iterator that contains all of the elements in this RDD. + * + * The iterator will consume as much memory as the largest partition in this RDD. + */ + def toLocalIterator(): JIterator[T] = { + import scala.collection.JavaConversions._ + rdd.toLocalIterator + } + + /** * Return an array that contains all of the elements in this RDD. * @deprecated As of Spark 1.0.0, toArray() is deprecated, use {@link #collect()} instead @@ -331,16 +368,20 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def count(): Long = rdd.count() /** - * (Experimental) Approximate version of count() that returns a potentially incomplete result + * :: Experimental :: + * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. */ + @Experimental def countApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = rdd.countApprox(timeout, confidence) /** - * (Experimental) Approximate version of count() that returns a potentially incomplete result + * :: Experimental :: + * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. */ + @Experimental def countApprox(timeout: Long): PartialResult[BoundedDouble] = rdd.countApprox(timeout) @@ -377,7 +418,10 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { new java.util.ArrayList(arr) } - def takeSample(withReplacement: Boolean, num: Int, seed: Int): JList[T] = { + def takeSample(withReplacement: Boolean, num: Int): JList[T] = + takeSample(withReplacement, num, Utils.random.nextLong) + + def takeSample(withReplacement: Boolean, num: Int, seed: Long): JList[T] = { import scala.collection.JavaConversions._ val arr: java.util.Collection[T] = rdd.takeSample(withReplacement, num, seed).toSeq new java.util.ArrayList(arr) @@ -519,12 +563,14 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return approximate number of distinct elements in the RDD. * - * The accuracy of approximation can be controlled through the relative standard deviation - * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vise versa. The default value of - * relativeSD is 0.05. + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @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 = rdd.countApproxDistinct(relativeSD) + def countApproxDistinct(relativeSD: Double): Long = rdd.countApproxDistinct(relativeSD) def name(): String = rdd.name 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 a2855d4db1d2e..1e0493c4855e0 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 @@ -22,6 +22,7 @@ import java.util.{Map => JMap} import scala.collection.JavaConversions import scala.collection.JavaConversions._ +import scala.language.implicitConversions import scala.reflect.ClassTag import com.google.common.base.Optional @@ -40,6 +41,12 @@ import org.apache.spark.rdd.RDD * [[org.apache.spark.api.java.JavaRDD]]s and works with Java collections instead of Scala ones. */ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWorkaround { + /** + * Create a JavaSparkContext that loads settings from system properties (for instance, when + * launching with ./bin/spark-submit). + */ + def this() = this(new SparkContext()) + /** * @param conf a [[org.apache.spark.SparkConf]] object specifying Spark parameters */ @@ -89,7 +96,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork */ def this(master: String, appName: String, sparkHome: String, jars: Array[String], environment: JMap[String, String]) = - this(new SparkContext(master, appName, sparkHome, jars.toSeq, environment)) + this(new SparkContext(master, appName, sparkHome, jars.toSeq, environment, Map())) private[spark] val env = sc.env @@ -108,9 +115,17 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork /** Default level of parallelism to use when not given by user (e.g. parallelize and makeRDD). */ def defaultParallelism: java.lang.Integer = sc.defaultParallelism - /** Default min number of partitions for Hadoop RDDs when not given by user */ + /** + * Default min number of partitions for Hadoop RDDs when not given by user. + * @deprecated As of Spark 1.0.0, defaultMinSplits is deprecated, use + * {@link #defaultMinPartitions()} instead + */ + @deprecated("use defaultMinPartitions", "1.0.0") def defaultMinSplits: java.lang.Integer = sc.defaultMinSplits + /** Default min number of partitions for Hadoop RDDs when not given by user */ + def defaultMinPartitions: java.lang.Integer = sc.defaultMinPartitions + /** Distribute a local Scala collection to form an RDD. */ def parallelize[T](list: java.util.List[T], numSlices: Int): JavaRDD[T] = { implicit val ctag: ClassTag[T] = fakeClassTag @@ -152,7 +167,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * 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, minSplits: Int): JavaRDD[String] = sc.textFile(path, minSplits) + def textFile(path: String, minPartitions: Int): JavaRDD[String] = + sc.textFile(path, minPartitions) /** * Read a directory of text files from HDFS, a local file system (available on all nodes), or any @@ -177,7 +193,19 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * (a-hdfs-path/part-nnnnn, its content) * }}} * - * @note Small files are preferred, as each file will be loaded fully in memory. + * @note Small files are preferred, large file is also allowable, but may cause bad performance. + * + * @param minPartitions A suggestion value of the minimal splitting number for input data. + */ + def wholeTextFiles(path: String, minPartitions: Int): JavaPairRDD[String, String] = + new JavaPairRDD(sc.wholeTextFiles(path, minPartitions)) + + /** + * Read a directory of text files from HDFS, a local file system (available on all nodes), or any + * Hadoop-supported file system URI. Each file is read as a single record and returned in a + * key-value pair, where the key is the path of each file, the value is the content of each file. + * + * @see `wholeTextFiles(path: String, minPartitions: Int)`. */ def wholeTextFiles(path: String): JavaPairRDD[String, String] = new JavaPairRDD(sc.wholeTextFiles(path)) @@ -192,11 +220,11 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V], - minSplits: Int + minPartitions: Int ): JavaPairRDD[K, V] = { implicit val ctagK: ClassTag[K] = ClassTag(keyClass) implicit val ctagV: ClassTag[V] = ClassTag(valueClass) - new JavaPairRDD(sc.sequenceFile(path, keyClass, valueClass, minSplits)) + new JavaPairRDD(sc.sequenceFile(path, keyClass, valueClass, minPartitions)) } /** Get an RDD for a Hadoop SequenceFile. @@ -220,9 +248,9 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * slow if you use the default serializer (Java serialization), though the nice thing about it is * that there's very little effort required to save arbitrary objects. */ - def objectFile[T](path: String, minSplits: Int): JavaRDD[T] = { + def objectFile[T](path: String, minPartitions: Int): JavaRDD[T] = { implicit val ctag: ClassTag[T] = fakeClassTag - sc.objectFile(path, minSplits)(ctag) + sc.objectFile(path, minPartitions)(ctag) } /** @@ -252,11 +280,11 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork inputFormatClass: Class[F], keyClass: Class[K], valueClass: Class[V], - minSplits: Int + minPartitions: Int ): JavaPairRDD[K, V] = { implicit val ctagK: ClassTag[K] = ClassTag(keyClass) implicit val ctagV: ClassTag[V] = ClassTag(valueClass) - new JavaPairRDD(sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass, minSplits)) + new JavaPairRDD(sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass, minPartitions)) } /** @@ -291,11 +319,11 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork inputFormatClass: Class[F], keyClass: Class[K], valueClass: Class[V], - minSplits: Int + minPartitions: Int ): JavaPairRDD[K, V] = { implicit val ctagK: ClassTag[K] = ClassTag(keyClass) implicit val ctagV: ClassTag[V] = ClassTag(valueClass) - new JavaPairRDD(sc.hadoopFile(path, inputFormatClass, keyClass, valueClass, minSplits)) + new JavaPairRDD(sc.hadoopFile(path, inputFormatClass, keyClass, valueClass, minPartitions)) } /** Get an RDD for a Hadoop file with an arbitrary InputFormat @@ -425,7 +453,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions. * The variable will be sent to each cluster only once. */ - def broadcast[T](value: T): Broadcast[T] = sc.broadcast(value) + def broadcast[T](value: T): Broadcast[T] = sc.broadcast(value)(fakeClassTag) /** Shut down the SparkContext. */ def stop() { @@ -548,6 +576,21 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * // In a separate thread: * sc.cancelJobGroup("some_job_to_cancel"); * }}} + * + * If interruptOnCancel is set to true for the job group, then job cancellation will result + * in Thread.interrupt() being called on the job's executor threads. This is useful to help ensure + * that the tasks are actually stopped in a timely manner, but is off by default due to HDFS-1208, + * where HDFS may respond to Thread.interrupt() by marking nodes as dead. + */ + def setJobGroup(groupId: String, description: String, interruptOnCancel: Boolean): Unit = + sc.setJobGroup(groupId, description, interruptOnCancel) + + /** + * Assigns a group ID to all the jobs started by this thread until the group ID is set to a + * different value or cleared. + * + * @see `setJobGroup(groupId: String, description: String, interruptThread: Boolean)`. + * This method sets interruptOnCancel to false. */ def setJobGroup(groupId: String, description: String): Unit = sc.setJobGroup(groupId, description) 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 ecbf18849ad48..22810cb1c662d 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 @@ -19,7 +19,7 @@ package org.apache.spark.api.java import com.google.common.base.Optional -object JavaUtils { +private[spark] object JavaUtils { def optionToOptional[T](option: Option[T]): Optional[T] = option match { case Some(value) => Optional.of(value) diff --git a/core/src/main/scala/org/apache/spark/api/java/package-info.java b/core/src/main/scala/org/apache/spark/api/java/package-info.java new file mode 100644 index 0000000000000..10a480fc78e44 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/java/package-info.java @@ -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. + */ + +/** + * Spark Java programming APIs. + */ +package org.apache.spark.api.java; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala new file mode 100644 index 0000000000000..adaa1ef6cf9ff --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.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.api.python + +import org.apache.spark.rdd.RDD +import org.apache.spark.Logging +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.io._ +import scala.util.{Failure, Success, Try} +import org.apache.spark.annotation.Experimental + + +/** + * :: Experimental :: + * A trait for use with reading custom classes in PySpark. Implement this trait and add custom + * transformation code by overriding the convert method. + */ +@Experimental +trait Converter[T, U] extends Serializable { + def convert(obj: T): U +} + +private[python] object Converter extends Logging { + + def getInstance(converterClass: Option[String]): Converter[Any, Any] = { + converterClass.map { cc => + Try { + val c = Class.forName(cc).newInstance().asInstanceOf[Converter[Any, Any]] + logInfo(s"Loaded converter: $cc") + c + } match { + case Success(c) => c + case Failure(err) => + logError(s"Failed to load converter: $cc") + throw err + } + }.getOrElse { new DefaultConverter } + } +} + +/** + * A converter that handles conversion of common [[org.apache.hadoop.io.Writable]] objects. + * Other objects are passed through without conversion. + */ +private[python] class DefaultConverter extends Converter[Any, Any] { + + /** + * Converts a [[org.apache.hadoop.io.Writable]] to the underlying primitive, String or + * object representation + */ + private def convertWritable(writable: Writable): Any = { + import collection.JavaConversions._ + writable match { + case iw: IntWritable => iw.get() + case dw: DoubleWritable => dw.get() + case lw: LongWritable => lw.get() + case fw: FloatWritable => fw.get() + case t: Text => t.toString + case bw: BooleanWritable => bw.get() + case byw: BytesWritable => byw.getBytes + case n: NullWritable => null + case aw: ArrayWritable => aw.get().map(convertWritable(_)) + case mw: MapWritable => mapAsJavaMap(mw.map { case (k, v) => + (convertWritable(k), convertWritable(v)) + }.toMap) + case other => other + } + } + + def convert(obj: Any): Any = { + obj match { + case writable: Writable => + convertWritable(writable) + case _ => + obj + } + } +} + +/** Utilities for working with Python objects <-> Hadoop-related objects */ +private[python] object PythonHadoopUtil { + + /** + * Convert a [[java.util.Map]] of properties to a [[org.apache.hadoop.conf.Configuration]] + */ + def mapToConf(map: java.util.Map[String, String]): Configuration = { + import collection.JavaConversions._ + val conf = new Configuration() + map.foreach{ case (k, v) => conf.set(k, v) } + conf + } + + /** + * Merges two configurations, returns a copy of left with keys from right overwriting + * any matching keys in left + */ + def mergeConfs(left: Configuration, right: Configuration): Configuration = { + import collection.JavaConversions._ + val copy = new Configuration(left) + right.iterator().foreach(entry => copy.set(entry.getKey, entry.getValue)) + copy + } + + /** + * Converts an RDD of key-value pairs, where key and/or value could be instances of + * [[org.apache.hadoop.io.Writable]], into an RDD[(K, V)] + */ + def convertRDD[K, V](rdd: RDD[(K, V)], + keyConverter: Converter[Any, Any], + valueConverter: Converter[Any, Any]): RDD[(Any, Any)] = { + rdd.map { case (k, v) => (keyConverter.convert(k), valueConverter.convert(v)) } + } + +} diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala index 95bec5030bfdd..e230d222b8604 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala @@ -50,4 +50,6 @@ private[spark] class PythonPartitioner( case _ => false } + + override def hashCode: Int = 31 * numPartitions + pyPartitionFunctionId.hashCode } 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 32f1100406d74..f6570d335757a 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 @@ -24,7 +24,13 @@ import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collectio import scala.collection.JavaConversions._ import scala.reflect.ClassTag +import scala.util.Try +import net.razorvine.pickle.{Pickler, Unpickler} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapred.{InputFormat, JobConf} +import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.spark._ import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} import org.apache.spark.broadcast.Broadcast @@ -51,63 +57,24 @@ private[spark] class PythonRDD[T: ClassTag]( override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = { val startTime = System.currentTimeMillis val env = SparkEnv.get - val worker = env.createPythonWorker(pythonExec, envVars.toMap) - - @volatile var readerException: Exception = null + val worker: Socket = env.createPythonWorker(pythonExec, envVars.toMap) // Start a thread to feed the process input from our parent's iterator - new Thread("stdin writer for " + pythonExec) { - override def run() { - try { - SparkEnv.set(env) - val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) - val dataOut = new DataOutputStream(stream) - // Partition index - dataOut.writeInt(split.index) - // sparkFilesDir - PythonRDD.writeUTF(SparkFiles.getRootDirectory, dataOut) - // Broadcast variables - dataOut.writeInt(broadcastVars.length) - for (broadcast <- broadcastVars) { - dataOut.writeLong(broadcast.id) - dataOut.writeInt(broadcast.value.length) - dataOut.write(broadcast.value) - } - // Python includes (*.zip and *.egg files) - dataOut.writeInt(pythonIncludes.length) - for (include <- pythonIncludes) { - PythonRDD.writeUTF(include, dataOut) - } - dataOut.flush() - // Serialized command: - dataOut.writeInt(command.length) - dataOut.write(command) - // Data values - PythonRDD.writeIteratorToStream(parent.iterator(split, context), dataOut) - dataOut.flush() - worker.shutdownOutput() - } catch { - case e: java.io.FileNotFoundException => - readerException = e - // Kill the Python worker process: - worker.shutdownOutput() - case e: IOException => - // This can happen for legitimate reasons if the Python code stops returning data - // before we are done passing elements through, e.g., for take(). Just log a message - // to say it happened. - logInfo("stdin writer to Python finished early") - logDebug("stdin writer to Python finished early", e) - } + val writerThread = new WriterThread(env, worker, split, context) + + context.addOnCompleteCallback { () => + writerThread.shutdownOnTaskCompletion() + + // Cleanup the worker socket. This will also cause the Python worker to exit. + try { + worker.close() + } catch { + case e: Exception => logWarning("Failed to close worker socket", e) } - }.start() + } - /* - * Partial fix for SPARK-1019: Attempts to stop reading the input stream since - * other completion callbacks might invalidate the input. Because interruption - * is not synchronous this still leaves a potential race where the interruption is - * processed only after the stream becomes invalid. - */ - context.addOnCompleteCallback(() => context.interrupted = true) + writerThread.start() + new MonitorThread(env, worker, context).start() // Return an iterator that read lines from the process's stdout val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream, bufferSize)) @@ -115,17 +82,14 @@ private[spark] class PythonRDD[T: ClassTag]( def next(): Array[Byte] = { val obj = _nextObj if (hasNext) { - // FIXME: can deadlock if worker is waiting for us to - // respond to current message (currently irrelevant because - // output is shutdown before we read any input) _nextObj = read() } obj } private def read(): Array[Byte] = { - if (readerException != null) { - throw readerException + if (writerThread.exception.isDefined) { + throw writerThread.exception.get } try { stream.readInt() match { @@ -133,6 +97,7 @@ private[spark] class PythonRDD[T: ClassTag]( val obj = new Array[Byte](length) stream.readFully(obj) obj + case 0 => Array.empty[Byte] case SpecialLengths.TIMING_DATA => // Timing data from worker val bootTime = stream.readLong() @@ -144,13 +109,14 @@ private[spark] class PythonRDD[T: ClassTag]( val total = finishTime - startTime logInfo("Times: total = %s, boot = %s, init = %s, finish = %s".format(total, boot, init, finish)) - read + read() case SpecialLengths.PYTHON_EXCEPTION_THROWN => // Signals that an exception has been thrown in python val exLength = stream.readInt() val obj = new Array[Byte](exLength) stream.readFully(obj) - throw new PythonException(new String(obj)) + throw new PythonException(new String(obj, "utf-8"), + writerThread.exception.getOrElse(null)) case SpecialLengths.END_OF_DATA_SECTION => // We've finished the data section of the output, but we can still // read some accumulator updates: @@ -160,30 +126,127 @@ private[spark] class PythonRDD[T: ClassTag]( val update = new Array[Byte](updateLen) stream.readFully(update) accumulator += Collections.singletonList(update) - } - Array.empty[Byte] + null } } catch { - case eof: EOFException => { + + case e: Exception if context.interrupted => + logDebug("Exception thrown after task interruption", e) + throw new TaskKilledException + + case e: Exception if writerThread.exception.isDefined => + logError("Python worker exited unexpectedly (crashed)", e) + logError("This may have been caused by a prior exception:", writerThread.exception.get) + throw writerThread.exception.get + + case eof: EOFException => throw new SparkException("Python worker exited unexpectedly (crashed)", eof) - } - case e: Throwable => throw e } } var _nextObj = read() - def hasNext = _nextObj.length != 0 + def hasNext = _nextObj != null } - stdoutIterator + new InterruptibleIterator(context, stdoutIterator) } val asJavaRDD : JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this) + + /** + * The thread responsible for writing the data from the PythonRDD's parent iterator to the + * Python process. + */ + class WriterThread(env: SparkEnv, worker: Socket, split: Partition, context: TaskContext) + extends Thread(s"stdout writer for $pythonExec") { + + @volatile private var _exception: Exception = null + + setDaemon(true) + + /** Contains the exception thrown while writing the parent iterator to the Python process. */ + def exception: Option[Exception] = Option(_exception) + + /** Terminates the writer thread, ignoring any exceptions that may occur due to cleanup. */ + def shutdownOnTaskCompletion() { + assert(context.completed) + this.interrupt() + } + + override def run(): Unit = Utils.logUncaughtExceptions { + try { + SparkEnv.set(env) + val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) + val dataOut = new DataOutputStream(stream) + // Partition index + dataOut.writeInt(split.index) + // sparkFilesDir + PythonRDD.writeUTF(SparkFiles.getRootDirectory, dataOut) + // Python includes (*.zip and *.egg files) + dataOut.writeInt(pythonIncludes.length) + for (include <- pythonIncludes) { + PythonRDD.writeUTF(include, dataOut) + } + // Broadcast variables + dataOut.writeInt(broadcastVars.length) + for (broadcast <- broadcastVars) { + dataOut.writeLong(broadcast.id) + dataOut.writeInt(broadcast.value.length) + dataOut.write(broadcast.value) + } + dataOut.flush() + // Serialized command: + dataOut.writeInt(command.length) + dataOut.write(command) + // Data values + PythonRDD.writeIteratorToStream(parent.iterator(split, context), dataOut) + dataOut.flush() + } catch { + case e: Exception if context.completed || context.interrupted => + logDebug("Exception thrown after task completion (likely due to cleanup)", e) + + case e: Exception => + // We must avoid throwing exceptions here, because the thread uncaught exception handler + // will kill the whole executor (see org.apache.spark.executor.Executor). + _exception = e + } finally { + Try(worker.shutdownOutput()) // kill Python worker process + } + } + } + + /** + * It is necessary to have a monitor thread for python workers if the user cancels with + * interrupts disabled. In that case we will need to explicitly kill the worker, otherwise the + * threads can block indefinitely. + */ + class MonitorThread(env: SparkEnv, worker: Socket, context: TaskContext) + extends Thread(s"Worker Monitor for $pythonExec") { + + setDaemon(true) + + override def run() { + // Kill the worker if it is interrupted, checking until task completion. + // TODO: This has a race condition if interruption occurs, as completed may still become true. + while (!context.interrupted && !context.completed) { + Thread.sleep(2000) + } + if (!context.completed) { + try { + logWarning("Incomplete task interrupted: Attempting to kill Python Worker") + env.destroyPythonWorker(pythonExec, envVars.toMap) + } catch { + case e: Exception => + logError("Exception when trying to kill worker", e) + } + } + } + } } /** Thrown for exceptions in user Python code. */ -private class PythonException(msg: String) extends Exception(msg) +private class PythonException(msg: String, cause: Exception) extends RuntimeException(msg, cause) /** * Form an RDD[(Array[Byte], Array[Byte])] from key-value pairs returned from Python. @@ -206,9 +269,29 @@ private object SpecialLengths { val TIMING_DATA = -3 } -private[spark] object PythonRDD { +private[spark] object PythonRDD extends Logging { val UTF8 = Charset.forName("UTF-8") + /** + * Adapter for calling SparkContext#runJob from Python. + * + * This method will return an iterator of an array that contains all elements in the RDD + * (effectively a collect()), but allows you to run on a certain subset of partitions, + * or to enable local execution. + */ + def runJob( + sc: SparkContext, + rdd: JavaRDD[Array[Byte]], + partitions: JArrayList[Int], + allowLocal: Boolean): Iterator[Array[Byte]] = { + type ByteArray = Array[Byte] + type UnrolledPartition = Array[ByteArray] + val allPartitions: Array[UnrolledPartition] = + sc.runJob(rdd, (x: Iterator[ByteArray]) => x.toArray, partitions, allowLocal) + val flattenedPartition: UnrolledPartition = Array.concat(allPartitions: _*) + flattenedPartition.iterator + } + def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: Int): JavaRDD[Array[Byte]] = { val file = new DataInputStream(new FileInputStream(filename)) @@ -222,7 +305,6 @@ private[spark] object PythonRDD { } } catch { case eof: EOFException => {} - case e: Throwable => throw e } JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) } @@ -267,6 +349,180 @@ private[spark] object PythonRDD { } } + /** + * Create an RDD from a path using [[org.apache.hadoop.mapred.SequenceFileInputFormat]], + * key and value class. + * A key and/or value converter class can optionally be passed in + * (see [[org.apache.spark.api.python.Converter]]) + */ + def sequenceFile[K, V]( + sc: JavaSparkContext, + path: String, + keyClassMaybeNull: String, + valueClassMaybeNull: String, + keyConverterClass: String, + valueConverterClass: String, + minSplits: Int) = { + val keyClass = Option(keyClassMaybeNull).getOrElse("org.apache.hadoop.io.Text") + val valueClass = Option(valueClassMaybeNull).getOrElse("org.apache.hadoop.io.Text") + implicit val kcm = ClassTag(Class.forName(keyClass)).asInstanceOf[ClassTag[K]] + implicit val vcm = ClassTag(Class.forName(valueClass)).asInstanceOf[ClassTag[V]] + val kc = kcm.runtimeClass.asInstanceOf[Class[K]] + val vc = vcm.runtimeClass.asInstanceOf[Class[V]] + + val rdd = sc.sc.sequenceFile[K, V](path, kc, vc, minSplits) + val keyConverter = Converter.getInstance(Option(keyConverterClass)) + val valueConverter = Converter.getInstance(Option(valueConverterClass)) + val converted = PythonHadoopUtil.convertRDD[K, V](rdd, keyConverter, valueConverter) + JavaRDD.fromRDD(SerDeUtil.rddToPython(converted)) + } + + /** + * Create an RDD from a file path, using an arbitrary [[org.apache.hadoop.mapreduce.InputFormat]], + * key and value class. + * A key and/or value converter class can optionally be passed in + * (see [[org.apache.spark.api.python.Converter]]) + */ + def newAPIHadoopFile[K, V, F <: NewInputFormat[K, V]]( + sc: JavaSparkContext, + path: String, + inputFormatClass: String, + keyClass: String, + valueClass: String, + keyConverterClass: String, + valueConverterClass: String, + confAsMap: java.util.HashMap[String, String]) = { + val conf = PythonHadoopUtil.mapToConf(confAsMap) + val baseConf = sc.hadoopConfiguration() + val mergedConf = PythonHadoopUtil.mergeConfs(baseConf, conf) + val rdd = + newAPIHadoopRDDFromClassNames[K, V, F](sc, + Some(path), inputFormatClass, keyClass, valueClass, mergedConf) + val keyConverter = Converter.getInstance(Option(keyConverterClass)) + val valueConverter = Converter.getInstance(Option(valueConverterClass)) + val converted = PythonHadoopUtil.convertRDD[K, V](rdd, keyConverter, valueConverter) + JavaRDD.fromRDD(SerDeUtil.rddToPython(converted)) + } + + /** + * Create an RDD from a [[org.apache.hadoop.conf.Configuration]] converted from a map that is + * passed in from Python, using an arbitrary [[org.apache.hadoop.mapreduce.InputFormat]], + * key and value class. + * A key and/or value converter class can optionally be passed in + * (see [[org.apache.spark.api.python.Converter]]) + */ + def newAPIHadoopRDD[K, V, F <: NewInputFormat[K, V]]( + sc: JavaSparkContext, + inputFormatClass: String, + keyClass: String, + valueClass: String, + keyConverterClass: String, + valueConverterClass: String, + confAsMap: java.util.HashMap[String, String]) = { + val conf = PythonHadoopUtil.mapToConf(confAsMap) + val rdd = + newAPIHadoopRDDFromClassNames[K, V, F](sc, + None, inputFormatClass, keyClass, valueClass, conf) + val keyConverter = Converter.getInstance(Option(keyConverterClass)) + val valueConverter = Converter.getInstance(Option(valueConverterClass)) + val converted = PythonHadoopUtil.convertRDD[K, V](rdd, keyConverter, valueConverter) + JavaRDD.fromRDD(SerDeUtil.rddToPython(converted)) + } + + private def newAPIHadoopRDDFromClassNames[K, V, F <: NewInputFormat[K, V]]( + sc: JavaSparkContext, + path: Option[String] = None, + inputFormatClass: String, + keyClass: String, + valueClass: String, + conf: Configuration) = { + implicit val kcm = ClassTag(Class.forName(keyClass)).asInstanceOf[ClassTag[K]] + implicit val vcm = ClassTag(Class.forName(valueClass)).asInstanceOf[ClassTag[V]] + implicit val fcm = ClassTag(Class.forName(inputFormatClass)).asInstanceOf[ClassTag[F]] + val kc = kcm.runtimeClass.asInstanceOf[Class[K]] + val vc = vcm.runtimeClass.asInstanceOf[Class[V]] + val fc = fcm.runtimeClass.asInstanceOf[Class[F]] + val rdd = if (path.isDefined) { + sc.sc.newAPIHadoopFile[K, V, F](path.get, fc, kc, vc, conf) + } else { + sc.sc.newAPIHadoopRDD[K, V, F](conf, fc, kc, vc) + } + rdd + } + + /** + * Create an RDD from a file path, using an arbitrary [[org.apache.hadoop.mapred.InputFormat]], + * key and value class. + * A key and/or value converter class can optionally be passed in + * (see [[org.apache.spark.api.python.Converter]]) + */ + def hadoopFile[K, V, F <: InputFormat[K, V]]( + sc: JavaSparkContext, + path: String, + inputFormatClass: String, + keyClass: String, + valueClass: String, + keyConverterClass: String, + valueConverterClass: String, + confAsMap: java.util.HashMap[String, String]) = { + val conf = PythonHadoopUtil.mapToConf(confAsMap) + val baseConf = sc.hadoopConfiguration() + val mergedConf = PythonHadoopUtil.mergeConfs(baseConf, conf) + val rdd = + hadoopRDDFromClassNames[K, V, F](sc, + Some(path), inputFormatClass, keyClass, valueClass, mergedConf) + val keyConverter = Converter.getInstance(Option(keyConverterClass)) + val valueConverter = Converter.getInstance(Option(valueConverterClass)) + val converted = PythonHadoopUtil.convertRDD[K, V](rdd, keyConverter, valueConverter) + JavaRDD.fromRDD(SerDeUtil.rddToPython(converted)) + } + + /** + * Create an RDD from a [[org.apache.hadoop.conf.Configuration]] converted from a map + * that is passed in from Python, using an arbitrary [[org.apache.hadoop.mapred.InputFormat]], + * key and value class + * A key and/or value converter class can optionally be passed in + * (see [[org.apache.spark.api.python.Converter]]) + */ + def hadoopRDD[K, V, F <: InputFormat[K, V]]( + sc: JavaSparkContext, + inputFormatClass: String, + keyClass: String, + valueClass: String, + keyConverterClass: String, + valueConverterClass: String, + confAsMap: java.util.HashMap[String, String]) = { + val conf = PythonHadoopUtil.mapToConf(confAsMap) + val rdd = + hadoopRDDFromClassNames[K, V, F](sc, + None, inputFormatClass, keyClass, valueClass, conf) + val keyConverter = Converter.getInstance(Option(keyConverterClass)) + val valueConverter = Converter.getInstance(Option(valueConverterClass)) + val converted = PythonHadoopUtil.convertRDD[K, V](rdd, keyConverter, valueConverter) + JavaRDD.fromRDD(SerDeUtil.rddToPython(converted)) + } + + private def hadoopRDDFromClassNames[K, V, F <: InputFormat[K, V]]( + sc: JavaSparkContext, + path: Option[String] = None, + inputFormatClass: String, + keyClass: String, + valueClass: String, + conf: Configuration) = { + implicit val kcm = ClassTag(Class.forName(keyClass)).asInstanceOf[ClassTag[K]] + implicit val vcm = ClassTag(Class.forName(valueClass)).asInstanceOf[ClassTag[V]] + implicit val fcm = ClassTag(Class.forName(inputFormatClass)).asInstanceOf[ClassTag[F]] + val kc = kcm.runtimeClass.asInstanceOf[Class[K]] + val vc = vcm.runtimeClass.asInstanceOf[Class[V]] + val fc = fcm.runtimeClass.asInstanceOf[Class[F]] + val rdd = if (path.isDefined) { + sc.sc.hadoopFile(path.get, fc, kc, vc) + } else { + sc.sc.hadoopRDD(new JobConf(conf), fc, kc, vc) + } + rdd + } + def writeUTF(str: String, dataOut: DataOutputStream) { val bytes = str.getBytes(UTF8) dataOut.writeInt(bytes.length) @@ -284,6 +540,36 @@ private[spark] object PythonRDD { file.close() } + /** + * Convert an RDD of serialized Python dictionaries to Scala Maps + * TODO: Support more Python types. + */ + def pythonToJavaMap(pyRDD: JavaRDD[Array[Byte]]): JavaRDD[Map[String, _]] = { + pyRDD.rdd.mapPartitions { iter => + val unpickle = new Unpickler + // TODO: Figure out why flatMap is necessay for pyspark + iter.flatMap { row => + unpickle.loads(row) match { + case objs: java.util.ArrayList[JMap[String, _] @unchecked] => objs.map(_.toMap) + // Incase the partition doesn't have a collection + case obj: JMap[String @unchecked, _] => Seq(obj.toMap) + } + } + } + } + + /** + * Convert and RDD of Java objects to and RDD of serialized Python objects, that is usable by + * PySpark. + */ + def javaToPython(jRDD: JavaRDD[Any]): JavaRDD[Array[Byte]] = { + jRDD.rdd.mapPartitions { iter => + val pickle = new Pickler + iter.map { row => + pickle.dumps(row) + } + } + } } private diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala new file mode 100644 index 0000000000000..6d3e257c4d5df --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.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.api.python + +import java.io.{File, InputStream, IOException, OutputStream} + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.SparkContext + +private[spark] object PythonUtils { + /** Get the PYTHONPATH for PySpark, either from SPARK_HOME, if it is set, or from our JAR */ + def sparkPythonPath: String = { + val pythonPath = new ArrayBuffer[String] + for (sparkHome <- sys.env.get("SPARK_HOME")) { + pythonPath += Seq(sparkHome, "python").mkString(File.separator) + pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.8.1-src.zip").mkString(File.separator) + } + pythonPath ++= SparkContext.jarOfObject(this) + pythonPath.mkString(File.pathSeparator) + } + + /** Merge PYTHONPATHS with the appropriate separator. Ignores blank strings. */ + def mergePythonPaths(paths: String*): String = { + paths.filter(_ != "").mkString(File.pathSeparator) + } +} + + +/** + * A utility class to redirect the child process's stdout or stderr. + */ +private[spark] class RedirectThread( + in: InputStream, + out: OutputStream, + name: String) + extends Thread(name) { + + setDaemon(true) + override def run() { + scala.util.control.Exception.ignoring(classOf[IOException]) { + // FIXME: We copy the stream on the level of bytes to avoid encoding problems. + val buf = new Array[Byte](1024) + var len = in.read(buf) + while (len != -1) { + out.write(buf, 0, len) + out.flush() + len = in.read(buf) + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index a5f0f3d5e7eae..759cbe2c46c52 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -17,15 +17,18 @@ package org.apache.spark.api.python -import java.io.{DataInputStream, File, IOException, OutputStreamWriter} +import java.io.{DataInputStream, InputStream, OutputStreamWriter} import java.net.{InetAddress, ServerSocket, Socket, SocketException} import scala.collection.JavaConversions._ import org.apache.spark._ +import org.apache.spark.util.Utils private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String, String]) - extends Logging { + extends Logging { + + import PythonWorkerFactory._ // Because forking processes from Java is expensive, we prefer to launch a single Python daemon // (pyspark/daemon.py) and tell it to fork new workers for our tasks. This daemon currently @@ -37,6 +40,11 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String val daemonHost = InetAddress.getByAddress(Array(127, 0, 0, 1)) var daemonPort: Int = 0 + val pythonPath = PythonUtils.mergePythonPaths( + PythonUtils.sparkPythonPath, + envVars.getOrElse("PYTHONPATH", ""), + sys.env.getOrElse("PYTHONPATH", "")) + def create(): Socket = { if (useDaemon) { createThroughDaemon() @@ -58,13 +66,11 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String try { new Socket(daemonHost, daemonPort) } catch { - case exc: SocketException => { + case exc: SocketException => logWarning("Python daemon unexpectedly quit, attempting to restart") stopDaemon() startDaemon() new Socket(daemonHost, daemonPort) - } - case e: Throwable => throw e } } } @@ -78,47 +84,14 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String serverSocket = new ServerSocket(0, 1, InetAddress.getByAddress(Array(127, 0, 0, 1))) // Create and start the worker - val sparkHome = new ProcessBuilder().environment().get("SPARK_HOME") - val pb = new ProcessBuilder(Seq(pythonExec, sparkHome + "/python/pyspark/worker.py")) + val pb = new ProcessBuilder(Seq(pythonExec, "-u", "-m", "pyspark.worker")) val workerEnv = pb.environment() workerEnv.putAll(envVars) - val pythonPath = sparkHome + "/python/" + File.pathSeparator + workerEnv.get("PYTHONPATH") workerEnv.put("PYTHONPATH", pythonPath) val worker = pb.start() - // Redirect the worker's stderr to ours - new Thread("stderr reader for " + pythonExec) { - setDaemon(true) - override def run() { - scala.util.control.Exception.ignoring(classOf[IOException]) { - // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - val in = worker.getErrorStream - val buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - System.err.write(buf, 0, len) - len = in.read(buf) - } - } - } - }.start() - - // Redirect worker's stdout to our stderr - new Thread("stdout reader for " + pythonExec) { - setDaemon(true) - override def run() { - scala.util.control.Exception.ignoring(classOf[IOException]) { - // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - val in = worker.getInputStream - val buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - System.err.write(buf, 0, len) - len = in.read(buf) - } - } - } - }.start() + // Redirect worker stdout and stderr + redirectStreamsToStderr(worker.getInputStream, worker.getErrorStream) // Tell the worker our port val out = new OutputStreamWriter(worker.getOutputStream) @@ -141,10 +114,6 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String null } - def stop() { - stopDaemon() - } - private def startDaemon() { synchronized { // Is it already running? @@ -154,54 +123,44 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String try { // Create and start the daemon - val sparkHome = new ProcessBuilder().environment().get("SPARK_HOME") - val pb = new ProcessBuilder(Seq(pythonExec, sparkHome + "/python/pyspark/daemon.py")) + val pb = new ProcessBuilder(Seq(pythonExec, "-u", "-m", "pyspark.daemon")) val workerEnv = pb.environment() workerEnv.putAll(envVars) - val pythonPath = sparkHome + "/python/" + File.pathSeparator + workerEnv.get("PYTHONPATH") workerEnv.put("PYTHONPATH", pythonPath) daemon = pb.start() - // Redirect the stderr to ours - new Thread("stderr reader for " + pythonExec) { - setDaemon(true) - override def run() { - scala.util.control.Exception.ignoring(classOf[IOException]) { - // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - val in = daemon.getErrorStream - val buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - System.err.write(buf, 0, len) - len = in.read(buf) - } - } - } - }.start() - val in = new DataInputStream(daemon.getInputStream) daemonPort = in.readInt() - // Redirect further stdout output to our stderr - new Thread("stdout reader for " + pythonExec) { - setDaemon(true) - override def run() { - scala.util.control.Exception.ignoring(classOf[IOException]) { - // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - val buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - System.err.write(buf, 0, len) - len = in.read(buf) - } - } - } - }.start() + // Redirect daemon stdout and stderr + redirectStreamsToStderr(in, daemon.getErrorStream) + } catch { - case e: Throwable => { + case e: Exception => + + // If the daemon exists, wait for it to finish and get its stderr + val stderr = Option(daemon) + .flatMap { d => Utils.getStderr(d, PROCESS_WAIT_TIMEOUT_MS) } + .getOrElse("") + stopDaemon() - throw e - } + + if (stderr != "") { + val formattedStderr = stderr.replace("\n", "\n ") + val errorMessage = s""" + |Error from python worker: + | $formattedStderr + |PYTHONPATH was: + | $pythonPath + |$e""" + + // Append error message from python daemon, but keep original stack trace + val wrappedException = new SparkException(errorMessage.stripMargin) + wrappedException.setStackTrace(e.getStackTrace) + throw wrappedException + } else { + throw e + } } // Important: don't close daemon's stdin (daemon.getOutputStream) so it can correctly @@ -209,6 +168,19 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } } + /** + * Redirect the given streams to our stderr in separate threads. + */ + private def redirectStreamsToStderr(stdout: InputStream, stderr: InputStream) { + try { + new RedirectThread(stdout, System.err, "stdout reader for " + pythonExec).start() + new RedirectThread(stderr, System.err, "stderr reader for " + pythonExec).start() + } catch { + case e: Exception => + logError("Exception in redirecting streams", e) + } + } + private def stopDaemon() { synchronized { // Request shutdown of existing daemon by sending SIGTERM @@ -220,4 +192,12 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String daemonPort = 0 } } + + def stop() { + stopDaemon() + } +} + +private object PythonWorkerFactory { + val PROCESS_WAIT_TIMEOUT_MS = 10000 } 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 new file mode 100644 index 0000000000000..9a012e7254901 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.python + +import scala.util.Try +import org.apache.spark.rdd.RDD +import org.apache.spark.Logging +import scala.util.Success +import scala.util.Failure +import net.razorvine.pickle.Pickler + + +/** Utilities for serialization / deserialization between Python and Java, using Pickle. */ +private[python] object SerDeUtil extends Logging { + + private def checkPickle(t: (Any, Any)): (Boolean, Boolean) = { + val pickle = new Pickler + val kt = Try { + pickle.dumps(t._1) + } + val vt = Try { + pickle.dumps(t._2) + } + (kt, vt) match { + case (Failure(kf), Failure(vf)) => + logWarning(s""" + |Failed to pickle Java object as key: ${t._1.getClass.getSimpleName}, falling back + |to 'toString'. Error: ${kf.getMessage}""".stripMargin) + logWarning(s""" + |Failed to pickle Java object as value: ${t._2.getClass.getSimpleName}, falling back + |to 'toString'. Error: ${vf.getMessage}""".stripMargin) + (true, true) + case (Failure(kf), _) => + logWarning(s""" + |Failed to pickle Java object as key: ${t._1.getClass.getSimpleName}, falling back + |to 'toString'. Error: ${kf.getMessage}""".stripMargin) + (true, false) + case (_, Failure(vf)) => + logWarning(s""" + |Failed to pickle Java object as value: ${t._2.getClass.getSimpleName}, falling back + |to 'toString'. Error: ${vf.getMessage}""".stripMargin) + (false, true) + case _ => + (false, false) + } + } + + /** + * Convert an RDD of key-value pairs to an RDD of serialized Python objects, that is usable + * by PySpark. By default, if serialization fails, toString is called and the string + * representation is serialized + */ + def rddToPython(rdd: RDD[(Any, Any)]): RDD[Array[Byte]] = { + val (keyFailed, valueFailed) = checkPickle(rdd.first()) + rdd.mapPartitions { iter => + val pickle = new Pickler + iter.map { case (k, v) => + if (keyFailed && valueFailed) { + pickle.dumps(Array(k.toString, v.toString)) + } else if (keyFailed) { + pickle.dumps(Array(k.toString, v)) + } else if (!keyFailed && valueFailed) { + pickle.dumps(Array(k, v.toString)) + } else { + pickle.dumps(Array(k, v)) + } + } + } + } + +} + 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 new file mode 100644 index 0000000000000..f0e3fb9aff5a0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.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.api.python + +import org.apache.spark.SparkContext +import org.apache.hadoop.io._ +import scala.Array +import java.io.{DataOutput, DataInput} +import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat +import org.apache.spark.api.java.JavaSparkContext + +/** + * A class to test MsgPack 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 setStr(str: String) { this.str = str } + def getInt = int + def setInt(int: Int) { this.int = int } + def getDouble = double + def setDouble(double: Double) { this.double = double } + + def write(out: DataOutput) = { + out.writeUTF(str) + out.writeInt(int) + out.writeDouble(double) + } + + def readFields(in: DataInput) = { + str = in.readUTF() + int = in.readInt() + double = in.readDouble() + } +} + +class TestConverter extends Converter[Any, Any] { + import collection.JavaConversions._ + override def convert(obj: Any) = { + val m = obj.asInstanceOf[MapWritable] + seqAsJavaList(m.keySet.map(w => w.asInstanceOf[DoubleWritable].get()).toSeq) + } +} + +/** + * This object contains method to generate SequenceFile test data and write it to a + * given directory (probably a temp directory) + */ +object WriteInputFormatTestDataGenerator { + import SparkContext._ + + def main(args: Array[String]) { + val path = args(0) + val sc = new JavaSparkContext("local[4]", "test-writables") + generateData(path, sc) + } + + def generateData(path: String, jsc: JavaSparkContext) { + val sc = jsc.sc + + val basePath = s"$path/sftestdata/" + val textPath = s"$basePath/sftext/" + val intPath = s"$basePath/sfint/" + val doublePath = s"$basePath/sfdouble/" + val arrPath = s"$basePath/sfarray/" + val mapPath = s"$basePath/sfmap/" + val classPath = s"$basePath/sfclass/" + val bytesPath = s"$basePath/sfbytes/" + val boolPath = s"$basePath/sfbool/" + val nullPath = s"$basePath/sfnull/" + + /* + * Create test data for IntWritable, DoubleWritable, Text, BytesWritable, + * BooleanWritable and NullWritable + */ + val intKeys = Seq((1, "aa"), (2, "bb"), (2, "aa"), (3, "cc"), (2, "bb"), (1, "aa")) + sc.parallelize(intKeys).saveAsSequenceFile(intPath) + sc.parallelize(intKeys.map{ case (k, v) => (k.toDouble, v) }).saveAsSequenceFile(doublePath) + sc.parallelize(intKeys.map{ case (k, v) => (k.toString, v) }).saveAsSequenceFile(textPath) + sc.parallelize(intKeys.map{ case (k, v) => (k, v.getBytes) }).saveAsSequenceFile(bytesPath) + val bools = Seq((1, true), (2, true), (2, false), (3, true), (2, false), (1, false)) + sc.parallelize(bools).saveAsSequenceFile(boolPath) + sc.parallelize(intKeys).map{ case (k, v) => + (new IntWritable(k), NullWritable.get()) + }.saveAsSequenceFile(nullPath) + + // Create test data for ArrayWritable + val data = Seq( + (1, Array(1.0, 2.0, 3.0)), + (2, Array(3.0, 4.0, 5.0)), + (3, Array(4.0, 5.0, 6.0)) + ) + sc.parallelize(data, numSlices = 2) + .map{ case (k, v) => + (new IntWritable(k), new ArrayWritable(classOf[DoubleWritable], v.map(new DoubleWritable(_)))) + }.saveAsNewAPIHadoopFile[SequenceFileOutputFormat[IntWritable, ArrayWritable]](arrPath) + + // Create test data for MapWritable, with keys DoubleWritable and values Text + val mapData = Seq( + (1, Map(2.0 -> "aa")), + (2, Map(3.0 -> "bb")), + (2, Map(1.0 -> "cc")), + (3, Map(2.0 -> "dd")), + (2, Map(1.0 -> "aa")), + (1, Map(3.0 -> "bb")) + ) + sc.parallelize(mapData, numSlices = 2).map{ case (i, m) => + val mw = new MapWritable() + val k = m.keys.head + val v = m.values.head + mw.put(new DoubleWritable(k), new Text(v)) + (new IntWritable(i), mw) + }.saveAsSequenceFile(mapPath) + + // Create test data for arbitrary custom writable TestWritable + val testClass = Seq( + ("1", TestWritable("test1", 123, 54.0)), + ("2", TestWritable("test2", 456, 8762.3)), + ("1", TestWritable("test3", 123, 423.1)), + ("3", TestWritable("test56", 456, 423.5)), + ("2", TestWritable("test2", 123, 5435.2)) + ) + val rdd = sc.parallelize(testClass, numSlices = 2).map{ case (k, v) => (new Text(k), v) } + rdd.saveAsNewAPIHadoopFile(classPath, + classOf[Text], classOf[TestWritable], + classOf[SequenceFileOutputFormat[Text, TestWritable]]) + } + + +} 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 e3c3a12d16f2a..76956f6a345d1 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -18,9 +18,10 @@ package org.apache.spark.broadcast import java.io.Serializable -import java.util.concurrent.atomic.AtomicLong -import org.apache.spark._ +import org.apache.spark.SparkException + +import scala.reflect.ClassTag /** * A broadcast variable. Broadcast variables allow the programmer to keep a read-only variable @@ -29,7 +30,8 @@ import org.apache.spark._ * attempts to distribute broadcast variables using efficient broadcast algorithms to reduce * communication cost. * - * Broadcast variables are created from a variable `v` by calling [[SparkContext#broadcast]]. + * Broadcast variables are created from a variable `v` by calling + * [[org.apache.spark.SparkContext#broadcast]]. * The broadcast variable is a wrapper around `v`, and its value can be accessed by calling the * `value` method. The interpreter session below shows this: * @@ -50,50 +52,81 @@ import org.apache.spark._ * @param id A unique identifier for the broadcast variable. * @tparam T Type of the data contained in the broadcast variable. */ -abstract class Broadcast[T](val id: Long) extends Serializable { - def value: T - - // We cannot have an abstract readObject here due to some weird issues with - // readObject having to be 'private' in sub-classes. - - override def toString = "Broadcast(" + id + ")" -} - -private[spark] -class BroadcastManager(val _isDriver: Boolean, conf: SparkConf, securityManager: SecurityManager) - extends Logging with Serializable { - - private var initialized = false - private var broadcastFactory: BroadcastFactory = null - - initialize() - - // Called by SparkContext or Executor before using Broadcast - private def initialize() { - synchronized { - if (!initialized) { - val broadcastFactoryClass = conf.get( - "spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") - - broadcastFactory = - Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory] +abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable { + + /** + * Flag signifying whether the broadcast variable is valid + * (that is, not already destroyed) or not. + */ + @volatile private var _isValid = true + + /** Get the broadcasted value. */ + def value: T = { + assertValid() + getValue() + } - // Initialize appropriate BroadcastFactory and BroadcastObject - broadcastFactory.initialize(isDriver, conf, securityManager) + /** + * Asynchronously delete cached copies of this broadcast on the executors. + * If the broadcast is used after this is called, it will need to be re-sent to each executor. + */ + def unpersist() { + unpersist(blocking = false) + } - initialized = true - } - } + /** + * Delete cached copies of this broadcast on the executors. If the broadcast is used after + * this is called, it will need to be re-sent to each executor. + * @param blocking Whether to block until unpersisting has completed + */ + def unpersist(blocking: Boolean) { + assertValid() + doUnpersist(blocking) } - def stop() { - broadcastFactory.stop() + /** + * Destroy all data and metadata related to this broadcast variable. Use this with caution; + * once a broadcast variable has been destroyed, it cannot be used again. + */ + private[spark] def destroy(blocking: Boolean) { + assertValid() + _isValid = false + doDestroy(blocking) } - private val nextBroadcastId = new AtomicLong(0) + /** + * Whether this Broadcast is actually usable. This should be false once persisted state is + * removed from the driver. + */ + private[spark] def isValid: Boolean = { + _isValid + } - def newBroadcast[T](value_ : T, isLocal: Boolean) = - broadcastFactory.newBroadcast[T](value_, isLocal, nextBroadcastId.getAndIncrement()) + /** + * Actually get the broadcasted value. Concrete implementations of Broadcast class must + * define their own way to get the value. + */ + private[spark] def getValue(): T + + /** + * Actually unpersist the broadcasted value on the executors. Concrete implementations of + * Broadcast class must define their own logic to unpersist their own data. + */ + private[spark] def doUnpersist(blocking: Boolean) + + /** + * Actually destroy all data and metadata related to this broadcast variable. + * Implementation of Broadcast class must define their own logic to destroy their own + * state. + */ + private[spark] def doDestroy(blocking: Boolean) + + /** Check if this broadcast is valid. If not valid, exception is thrown. */ + private[spark] def assertValid() { + if (!_isValid) { + throw new SparkException("Attempted to use %s after it has been destroyed!".format(toString)) + } + } - def isDriver = _isDriver + override def toString = "Broadcast(" + id + ")" } diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala index 6beecaeced5be..a8c827030a1ef 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -16,18 +16,24 @@ */ package org.apache.spark.broadcast -import org.apache.spark.SecurityManager +import scala.reflect.ClassTag + +import org.apache.spark.SecurityManager import org.apache.spark.SparkConf +import org.apache.spark.annotation.DeveloperApi /** - * An interface for all the broadcast implementations in Spark (to allow + * :: DeveloperApi :: + * An interface for all the broadcast implementations in Spark (to allow * multiple broadcast implementations). SparkContext uses a user-specified * BroadcastFactory implementation to instantiate a particular broadcast for the * entire Spark job. */ +@DeveloperApi trait BroadcastFactory { - def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit - def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T] + def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit + def newBroadcast[T: ClassTag](value: T, isLocal: Boolean, id: Long): Broadcast[T] + def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean): Unit def stop(): Unit } diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala new file mode 100644 index 0000000000000..c88be6aba6901 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.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.broadcast + +import java.util.concurrent.atomic.AtomicLong + +import scala.reflect.ClassTag + +import org.apache.spark._ + +private[spark] class BroadcastManager( + val isDriver: Boolean, + conf: SparkConf, + securityManager: SecurityManager) + extends Logging { + + private var initialized = false + private var broadcastFactory: BroadcastFactory = null + + initialize() + + // Called by SparkContext or Executor before using Broadcast + private def initialize() { + synchronized { + if (!initialized) { + val broadcastFactoryClass = + conf.get("spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") + + broadcastFactory = + Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory] + + // Initialize appropriate BroadcastFactory and BroadcastObject + broadcastFactory.initialize(isDriver, conf, securityManager) + + initialized = true + } + } + } + + def stop() { + broadcastFactory.stop() + } + + private val nextBroadcastId = new AtomicLong(0) + + def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean) = { + broadcastFactory.newBroadcast[T](value_, isLocal, nextBroadcastId.getAndIncrement()) + } + + def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean) { + broadcastFactory.unbroadcast(id, removeFromDriver, blocking) + } +} 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 e8eb04bb10469..4f6cabaff2b99 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -17,34 +17,67 @@ package org.apache.spark.broadcast -import java.io.{File, FileOutputStream, ObjectInputStream, OutputStream} +import java.io.{File, FileOutputStream, ObjectInputStream, ObjectOutputStream, OutputStream} +import java.io.{BufferedInputStream, BufferedOutputStream} import java.net.{URL, URLConnection, URI} import java.util.concurrent.TimeUnit -import it.unimi.dsi.fastutil.io.FastBufferedInputStream -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream +import scala.reflect.ClassTag -import org.apache.spark.{SparkConf, HttpServer, Logging, SecurityManager, SparkEnv} +import org.apache.spark.{HttpServer, Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.io.CompressionCodec import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashSet, Utils} -private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) +/** + * A [[org.apache.spark.broadcast.Broadcast]] implementation that uses HTTP server + * as a broadcast mechanism. The first time a HTTP broadcast variable (sent as part of a + * task) is deserialized in the executor, the broadcasted data is fetched from the driver + * (through a HTTP server running at the driver) and stored in the BlockManager of the + * executor to speed up future accesses. + */ +private[spark] class HttpBroadcast[T: ClassTag]( + @transient var value_ : T, isLocal: Boolean, id: Long) extends Broadcast[T](id) with Logging with Serializable { - def value = value_ + def getValue = value_ - def blockId = BroadcastBlockId(id) + val blockId = BroadcastBlockId(id) + /* + * Broadcasted data is also stored in the BlockManager of the driver. The BlockManagerMaster + * does not need to be told about this block as not only need to know about this data block. + */ HttpBroadcast.synchronized { - SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false) + SparkEnv.get.blockManager.putSingle( + blockId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) } if (!isLocal) { HttpBroadcast.write(id, value_) } - // Called by JVM when deserializing an object + /** + * Remove all persisted state associated with this HTTP broadcast on the executors. + */ + def doUnpersist(blocking: Boolean) { + HttpBroadcast.unpersist(id, removeFromDriver = false, blocking) + } + + /** + * Remove all persisted state associated with this HTTP broadcast on the executors and driver. + */ + def doDestroy(blocking: Boolean) { + HttpBroadcast.unpersist(id, removeFromDriver = true, blocking) + } + + /** Used by the JVM when serializing this object. */ + private def writeObject(out: ObjectOutputStream) { + assertValid() + out.defaultWriteObject() + } + + /** Used by the JVM when deserializing this object. */ private def readObject(in: ObjectInputStream) { in.defaultReadObject() HttpBroadcast.synchronized { @@ -54,7 +87,13 @@ private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolea logInfo("Started reading broadcast variable " + id) val start = System.nanoTime value_ = HttpBroadcast.read[T](id) - SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false) + /* + * We cache broadcast data in the BlockManager so that subsequent tasks using it + * do not need to re-fetch. This data is only used locally and no other node + * needs to fetch this block, so we don't notify the master. + */ + SparkEnv.get.blockManager.putSingle( + blockId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) val time = (System.nanoTime - start) / 1e9 logInfo("Reading broadcast variable " + id + " took " + time + " s") } @@ -63,23 +102,8 @@ private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolea } } -/** - * A [[BroadcastFactory]] implementation that uses a HTTP server as the broadcast medium. - */ -class HttpBroadcastFactory extends BroadcastFactory { - def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { - HttpBroadcast.initialize(isDriver, conf, securityMgr) - } - - def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = - new HttpBroadcast[T](value_, isLocal, id) - - def stop() { HttpBroadcast.stop() } -} - -private object HttpBroadcast extends Logging { +private[spark] object HttpBroadcast extends Logging { private var initialized = false - private var broadcastDir: File = null private var compress: Boolean = false private var bufferSize: Int = 65536 @@ -88,12 +112,10 @@ private object HttpBroadcast extends Logging { private var securityManager: SecurityManager = null // TODO: This shouldn't be a global variable so that multiple SparkContexts can coexist - private val files = new TimeStampedHashSet[String] - private var cleaner: MetadataCleaner = null - + private val files = new TimeStampedHashSet[File] private val httpReadTimeout = TimeUnit.MILLISECONDS.convert(5, TimeUnit.MINUTES).toInt - private var compressionCodec: CompressionCodec = null + private var cleaner: MetadataCleaner = null def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { synchronized { @@ -136,23 +158,25 @@ private object HttpBroadcast extends Logging { logInfo("Broadcast server started at " + serverUri) } + def getFile(id: Long) = new File(broadcastDir, BroadcastBlockId(id).name) + def write(id: Long, value: Any) { - val file = new File(broadcastDir, BroadcastBlockId(id).name) + val file = getFile(id) val out: OutputStream = { if (compress) { compressionCodec.compressedOutputStream(new FileOutputStream(file)) } else { - new FastBufferedOutputStream(new FileOutputStream(file), bufferSize) + new BufferedOutputStream(new FileOutputStream(file), bufferSize) } } val ser = SparkEnv.get.serializer.newInstance() val serOut = ser.serializeStream(out) serOut.writeObject(value) serOut.close() - files += file.getAbsolutePath + files += file } - def read[T](id: Long): T = { + def read[T: ClassTag](id: Long): T = { logDebug("broadcast read server: " + serverUri + " id: broadcast-" + id) val url = serverUri + "/" + BroadcastBlockId(id).name @@ -160,7 +184,7 @@ private object HttpBroadcast extends Logging { if (securityManager.isAuthenticationEnabled()) { logDebug("broadcast security enabled") val newuri = Utils.constructURIForAuthentication(new URI(url), securityManager) - uc = newuri.toURL().openConnection() + uc = newuri.toURL.openConnection() uc.setAllowUserInteraction(false) } else { logDebug("broadcast not using security") @@ -169,11 +193,11 @@ private object HttpBroadcast extends Logging { val in = { uc.setReadTimeout(httpReadTimeout) - val inputStream = uc.getInputStream(); + val inputStream = uc.getInputStream if (compress) { compressionCodec.compressedInputStream(inputStream) } else { - new FastBufferedInputStream(inputStream, bufferSize) + new BufferedInputStream(inputStream, bufferSize) } } val ser = SparkEnv.get.serializer.newInstance() @@ -183,20 +207,48 @@ private object HttpBroadcast extends Logging { obj } - def cleanup(cleanupTime: Long) { + /** + * Remove all persisted blocks associated with this HTTP broadcast on the executors. + * 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 { + SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) + if (removeFromDriver) { + val file = getFile(id) + files.remove(file) + deleteBroadcastFile(file) + } + } + + /** + * Periodically clean up old broadcasts by removing the associated map entries and + * deleting the associated files. + */ + private def cleanup(cleanupTime: Long) { val iterator = files.internalMap.entrySet().iterator() while(iterator.hasNext) { val entry = iterator.next() val (file, time) = (entry.getKey, entry.getValue) if (time < cleanupTime) { - try { - iterator.remove() - new File(file.toString).delete() - logInfo("Deleted broadcast file '" + file + "'") - } catch { - case e: Exception => logWarning("Could not delete broadcast file '" + file + "'", e) + iterator.remove() + deleteBroadcastFile(file) + } + } + } + + private def deleteBroadcastFile(file: File) { + try { + if (file.exists) { + if (file.delete()) { + logInfo("Deleted broadcast file: %s".format(file)) + } else { + logWarning("Could not delete broadcast file: %s".format(file)) } } + } catch { + case e: Exception => + logError("Exception while deleting broadcast file: %s".format(file), e) } } } diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala new file mode 100644 index 0000000000000..d5a031e2bbb59 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.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.broadcast + +import scala.reflect.ClassTag + +import org.apache.spark.{SecurityManager, SparkConf} + +/** + * A [[org.apache.spark.broadcast.BroadcastFactory]] implementation that uses a + * HTTP server as the broadcast mechanism. Refer to + * [[org.apache.spark.broadcast.HttpBroadcast]] for more details about this mechanism. + */ +class HttpBroadcastFactory extends BroadcastFactory { + def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { + HttpBroadcast.initialize(isDriver, conf, securityMgr) + } + + def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) = + new HttpBroadcast[T](value_, isLocal, id) + + def stop() { HttpBroadcast.stop() } + + /** + * Remove all persisted state associated with the HTTP broadcast with the given ID. + * @param removeFromDriver Whether to remove state from the driver + * @param blocking Whether to block until unbroadcasted + */ + def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean) { + HttpBroadcast.unpersist(id, removeFromDriver, blocking) + } +} 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 2595c15104e87..734de37ba115d 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -17,24 +17,45 @@ package org.apache.spark.broadcast -import java.io._ +import java.io.{ByteArrayInputStream, ObjectInputStream, ObjectOutputStream} +import scala.reflect.ClassTag import scala.math import scala.util.Random -import org.apache.spark._ -import org.apache.spark.storage.{BroadcastBlockId, BroadcastHelperBlockId, StorageLevel} +import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException} +import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} import org.apache.spark.util.Utils -private[spark] class TorrentBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) -extends Broadcast[T](id) with Logging with Serializable { +/** + * A [[org.apache.spark.broadcast.Broadcast]] implementation that uses a BitTorrent-like + * protocol to do a distributed transfer of the broadcasted data to the executors. + * The mechanism is as follows. The driver divides the serializes the broadcasted data, + * divides it into smaller chunks, and stores them in the BlockManager of the driver. + * These chunks are reported to the BlockManagerMaster so that all the executors can + * learn the location of those chunks. The first time the broadcast variable (sent as + * part of task) is deserialized at a executor, all the chunks are fetched using + * the BlockManager. When all the chunks are fetched (initially from the driver's + * BlockManager), they are combined and deserialized to recreate the broadcasted data. + * However, the chunks are also stored in the BlockManager and reported to the + * BlockManagerMaster. As more executors fetch the chunks, BlockManagerMaster learns + * multiple locations for each chunk. Hence, subsequent fetches of each chunk will be + * made to other executors who already have those chunks, resulting in a distributed + * fetching. This prevents the driver from being the bottleneck in sending out multiple + * copies of the broadcast data (one per executor) as done by the + * [[org.apache.spark.broadcast.HttpBroadcast]]. + */ +private[spark] class TorrentBroadcast[T: ClassTag]( + @transient var value_ : T, isLocal: Boolean, id: Long) + extends Broadcast[T](id) with Logging with Serializable { - def value = value_ + def getValue = value_ - def broadcastId = BroadcastBlockId(id) + val broadcastId = BroadcastBlockId(id) TorrentBroadcast.synchronized { - SparkEnv.get.blockManager.putSingle(broadcastId, value_, StorageLevel.MEMORY_AND_DISK, false) + SparkEnv.get.blockManager.putSingle( + broadcastId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) } @transient var arrayOfBlocks: Array[TorrentBlock] = null @@ -46,32 +67,52 @@ extends Broadcast[T](id) with Logging with Serializable { sendBroadcast() } - def sendBroadcast() { - var tInfo = TorrentBroadcast.blockifyObject(value_) + /** + * Remove all persisted state associated with this Torrent broadcast on the executors. + */ + def doUnpersist(blocking: Boolean) { + TorrentBroadcast.unpersist(id, removeFromDriver = false, blocking) + } + + /** + * Remove all persisted state associated with this Torrent broadcast on the executors + * and driver. + */ + def doDestroy(blocking: Boolean) { + TorrentBroadcast.unpersist(id, removeFromDriver = true, blocking) + } + def sendBroadcast() { + val tInfo = TorrentBroadcast.blockifyObject(value_) totalBlocks = tInfo.totalBlocks totalBytes = tInfo.totalBytes hasBlocks = tInfo.totalBlocks // Store meta-info - val metaId = BroadcastHelperBlockId(broadcastId, "meta") + val metaId = BroadcastBlockId(id, "meta") val metaInfo = TorrentInfo(null, totalBlocks, totalBytes) TorrentBroadcast.synchronized { SparkEnv.get.blockManager.putSingle( - metaId, metaInfo, StorageLevel.MEMORY_AND_DISK, true) + metaId, metaInfo, StorageLevel.MEMORY_AND_DISK, tellMaster = true) } // Store individual pieces for (i <- 0 until totalBlocks) { - val pieceId = BroadcastHelperBlockId(broadcastId, "piece" + i) + val pieceId = BroadcastBlockId(id, "piece" + i) TorrentBroadcast.synchronized { SparkEnv.get.blockManager.putSingle( - pieceId, tInfo.arrayOfBlocks(i), StorageLevel.MEMORY_AND_DISK, true) + pieceId, tInfo.arrayOfBlocks(i), StorageLevel.MEMORY_AND_DISK, tellMaster = true) } } } - // Called by JVM when deserializing an object + /** Used by the JVM when serializing this object. */ + private def writeObject(out: ObjectOutputStream) { + assertValid() + out.defaultWriteObject() + } + + /** Used by the JVM when deserializing this object. */ private def readObject(in: ObjectInputStream) { in.defaultReadObject() TorrentBroadcast.synchronized { @@ -86,18 +127,22 @@ extends Broadcast[T](id) with Logging with Serializable { // Initialize @transient variables that will receive garbage values from the master. resetWorkerVariables() - if (receiveBroadcast(id)) { + if (receiveBroadcast()) { value_ = TorrentBroadcast.unBlockifyObject[T](arrayOfBlocks, totalBytes, totalBlocks) - // Store the merged copy in cache so that the next worker doesn't need to rebuild it. - // This creates a tradeoff between memory usage and latency. - // Storing copy doubles the memory footprint; not storing doubles deserialization cost. + /* Store the merged copy in cache so that the next worker doesn't need to rebuild it. + * This creates a trade-off between memory usage and latency. Storing copy doubles + * the memory footprint; not storing doubles deserialization cost. Also, + * this does not need to be reported to BlockManagerMaster since other executors + * does not need to access this block (they only need to fetch the chunks, + * which are reported). + */ SparkEnv.get.blockManager.putSingle( - broadcastId, value_, StorageLevel.MEMORY_AND_DISK, false) + broadcastId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) // Remove arrayOfBlocks from memory once value_ is on local cache resetWorkerVariables() - } else { + } else { logError("Reading broadcast variable " + id + " failed") } @@ -114,9 +159,10 @@ extends Broadcast[T](id) with Logging with Serializable { hasBlocks = 0 } - def receiveBroadcast(variableID: Long): Boolean = { - // Receive meta-info - val metaId = BroadcastHelperBlockId(broadcastId, "meta") + def receiveBroadcast(): Boolean = { + // Receive meta-info about the size of broadcast data, + // the number of chunks it is divided into, etc. + val metaId = BroadcastBlockId(id, "meta") var attemptId = 10 while (attemptId > 0 && totalBlocks == -1) { TorrentBroadcast.synchronized { @@ -138,17 +184,21 @@ extends Broadcast[T](id) with Logging with Serializable { return false } - // Receive actual blocks + /* + * Fetch actual chunks of data. Note that all these chunks are stored in + * the BlockManager and reported to the master, so that other executors + * can find out and pull the chunks from this executor. + */ val recvOrder = new Random().shuffle(Array.iterate(0, totalBlocks)(_ + 1).toList) for (pid <- recvOrder) { - val pieceId = BroadcastHelperBlockId(broadcastId, "piece" + pid) + val pieceId = BroadcastBlockId(id, "piece" + pid) TorrentBroadcast.synchronized { SparkEnv.get.blockManager.getSingle(pieceId) match { case Some(x) => arrayOfBlocks(pid) = x.asInstanceOf[TorrentBlock] hasBlocks += 1 SparkEnv.get.blockManager.putSingle( - pieceId, arrayOfBlocks(pid), StorageLevel.MEMORY_AND_DISK, true) + pieceId, arrayOfBlocks(pid), StorageLevel.MEMORY_AND_DISK, tellMaster = true) case None => throw new SparkException("Failed to get " + pieceId + " of " + broadcastId) @@ -156,16 +206,16 @@ extends Broadcast[T](id) with Logging with Serializable { } } - (hasBlocks == totalBlocks) + hasBlocks == totalBlocks } } -private object TorrentBroadcast -extends Logging { - +private[spark] object TorrentBroadcast extends Logging { + private lazy val BLOCK_SIZE = conf.getInt("spark.broadcast.blockSize", 4096) * 1024 private var initialized = false private var conf: SparkConf = null + def initialize(_isDriver: Boolean, conf: SparkConf) { TorrentBroadcast.conf = conf // TODO: we might have to fix it in tests synchronized { @@ -179,39 +229,37 @@ extends Logging { initialized = false } - lazy val BLOCK_SIZE = conf.getInt("spark.broadcast.blockSize", 4096) * 1024 - def blockifyObject[T](obj: T): TorrentInfo = { val byteArray = Utils.serialize[T](obj) val bais = new ByteArrayInputStream(byteArray) - var blockNum = (byteArray.length / BLOCK_SIZE) + var blockNum = byteArray.length / BLOCK_SIZE if (byteArray.length % BLOCK_SIZE != 0) { blockNum += 1 } - var retVal = new Array[TorrentBlock](blockNum) - var blockID = 0 + val blocks = new Array[TorrentBlock](blockNum) + var blockId = 0 for (i <- 0 until (byteArray.length, BLOCK_SIZE)) { val thisBlockSize = math.min(BLOCK_SIZE, byteArray.length - i) - var tempByteArray = new Array[Byte](thisBlockSize) - val hasRead = bais.read(tempByteArray, 0, thisBlockSize) + val tempByteArray = new Array[Byte](thisBlockSize) + bais.read(tempByteArray, 0, thisBlockSize) - retVal(blockID) = new TorrentBlock(blockID, tempByteArray) - blockID += 1 + blocks(blockId) = new TorrentBlock(blockId, tempByteArray) + blockId += 1 } bais.close() - val tInfo = TorrentInfo(retVal, blockNum, byteArray.length) - tInfo.hasBlocks = blockNum - - tInfo + val info = TorrentInfo(blocks, blockNum, byteArray.length) + info.hasBlocks = blockNum + info } - def unBlockifyObject[T](arrayOfBlocks: Array[TorrentBlock], - totalBytes: Int, - totalBlocks: Int): T = { + def unBlockifyObject[T]( + arrayOfBlocks: Array[TorrentBlock], + totalBytes: Int, + totalBlocks: Int): T = { val retByteArray = new Array[Byte](totalBytes) for (i <- 0 until totalBlocks) { System.arraycopy(arrayOfBlocks(i).byteArray, 0, retByteArray, @@ -220,6 +268,13 @@ extends Logging { Utils.deserialize[T](retByteArray, Thread.currentThread.getContextClassLoader) } + /** + * 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) = synchronized { + SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) + } } private[spark] case class TorrentBlock( @@ -228,25 +283,10 @@ private[spark] case class TorrentBlock( extends Serializable private[spark] case class TorrentInfo( - @transient arrayOfBlocks : Array[TorrentBlock], + @transient arrayOfBlocks: Array[TorrentBlock], totalBlocks: Int, totalBytes: Int) extends Serializable { @transient var hasBlocks = 0 } - -/** - * A [[BroadcastFactory]] that creates a torrent-based implementation of broadcast. - */ -class TorrentBroadcastFactory extends BroadcastFactory { - - def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { - TorrentBroadcast.initialize(isDriver, conf) - } - - def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = - new TorrentBroadcast[T](value_, isLocal, id) - - def stop() { TorrentBroadcast.stop() } -} diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala new file mode 100644 index 0000000000000..1de8396a0e17f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.broadcast + +import scala.reflect.ClassTag + +import org.apache.spark.{SecurityManager, SparkConf} + +/** + * A [[org.apache.spark.broadcast.Broadcast]] implementation that uses a BitTorrent-like + * protocol to do a distributed transfer of the broadcasted data to the executors. Refer to + * [[org.apache.spark.broadcast.TorrentBroadcast]] for more details. + */ +class TorrentBroadcastFactory extends BroadcastFactory { + + def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { + TorrentBroadcast.initialize(isDriver, conf) + } + + def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) = + new TorrentBroadcast[T](value_, isLocal, id) + + def stop() { TorrentBroadcast.stop() } + + /** + * Remove all persisted state associated with the torrent broadcast with the given ID. + * @param removeFromDriver Whether to remove state from the driver. + * @param blocking Whether to block until unbroadcasted + */ + def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean) { + TorrentBroadcast.unpersist(id, removeFromDriver, blocking) + } +} diff --git a/core/src/main/scala/org/apache/spark/broadcast/package-info.java b/core/src/main/scala/org/apache/spark/broadcast/package-info.java new file mode 100644 index 0000000000000..1510e6e84c7a8 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/broadcast/package-info.java @@ -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. + */ + +/** + * Spark's broadcast variables, used to broadcast immutable datasets to all nodes. + */ +package org.apache.spark.broadcast; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/broadcast/package.scala b/core/src/main/scala/org/apache/spark/broadcast/package.scala index 01bf88629a7dd..4f2c3c07161a3 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/package.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/package.scala @@ -18,7 +18,7 @@ package org.apache.spark /** - * Package for broadcast variables. See [[broadcast.Broadcast]] for details. + * Spark's broadcast variables, used to broadcast immutable datasets to all nodes. */ package object broadcast { // For package docs only 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 15fa8a7679874..86305d2ea8a09 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -17,8 +17,6 @@ package org.apache.spark.deploy -import org.apache.spark.scheduler.EventLoggingInfo - private[spark] class ApplicationDescription( val name: String, val maxCores: Option[Int], @@ -26,7 +24,7 @@ private[spark] class ApplicationDescription( val command: Command, val sparkHome: Option[String], var appUiUrl: String, - val eventLogInfo: Option[EventLoggingInfo] = None) + val eventLogDir: Option[String] = None) extends Serializable { val user = System.getProperty("user.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 8fd2c7e95b966..c371dc3a51c73 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -54,8 +54,21 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends System.getenv().foreach{case (k, v) => env(k) = v} val mainClass = "org.apache.spark.deploy.worker.DriverWrapper" + + val classPathConf = "spark.driver.extraClassPath" + val classPathEntries = sys.props.get(classPathConf).toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } + + val libraryPathConf = "spark.driver.extraLibraryPath" + val libraryPathEntries = sys.props.get(libraryPathConf).toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } + + val javaOptionsConf = "spark.driver.extraJavaOptions" + val javaOpts = sys.props.get(javaOptionsConf) val command = new Command(mainClass, Seq("{{WORKER_URL}}", driverArgs.mainClass) ++ - driverArgs.driverOptions, env) + driverArgs.driverOptions, env, classPathEntries, libraryPathEntries, javaOpts) val driverDescription = new DriverDescription( driverArgs.jarUrl, @@ -68,7 +81,7 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends case "kill" => val driverId = driverArgs.driverId - val killFuture = masterActor ! RequestKillDriver(driverId) + masterActor ! RequestKillDriver(driverId) } } @@ -144,7 +157,7 @@ object Client { // TODO: See if we can initialize akka so return messages are sent back using the same TCP // flow. Else, this (sadly) requires the DriverClient be routable from the Master. val (actorSystem, _) = AkkaUtils.createActorSystem( - "driverClient", Utils.localHostName(), 0, false, conf, new SecurityManager(conf)) + "driverClient", Utils.localHostName(), 0, conf, new SecurityManager(conf)) actorSystem.actorOf(Props(classOf[ClientActor], driverArgs, conf)) 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 c07838f798799..39150deab863c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -21,6 +21,8 @@ import scala.collection.mutable.ListBuffer import org.apache.log4j.Level +import org.apache.spark.util.MemoryParam + /** * Command-line parser for the driver client. */ @@ -43,7 +45,7 @@ private[spark] class ClientArguments(args: Array[String]) { // kill parameters var driverId: String = "" - + parse(args.toList) def parse(args: List[String]): Unit = args match { @@ -51,8 +53,8 @@ private[spark] class ClientArguments(args: Array[String]) { cores = value.toInt parse(tail) - case ("--memory" | "-m") :: value :: tail => - memory = value.toInt + case ("--memory" | "-m") :: MemoryParam(value) :: tail => + memory = value parse(tail) case ("--supervise" | "-s") :: tail => diff --git a/core/src/main/scala/org/apache/spark/deploy/Command.scala b/core/src/main/scala/org/apache/spark/deploy/Command.scala index fa8af9a646750..32f3ba385084f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Command.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Command.scala @@ -22,5 +22,8 @@ import scala.collection.Map private[spark] case class Command( mainClass: String, arguments: Seq[String], - environment: Map[String, String]) { + environment: Map[String, String], + classPathEntries: Seq[String], + libraryPathEntries: Seq[String], + extraJavaOptions: Option[String] = None) { } 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 83ce14a0a806a..a7368f9f3dfbe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -86,6 +86,10 @@ private[deploy] object DeployMessages { case class KillDriver(driverId: String) extends DeployMessage + // Worker internal + + case object WorkDirCleanup // Sent to Worker actor periodically for cleaning up app folders + // AppClient to Master case class RegisterApplication(appDescription: ApplicationDescription) diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala index 37dfa7fec0831..9f34d01e6db48 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala @@ -19,9 +19,9 @@ package org.apache.spark.deploy private[spark] object ExecutorState extends Enumeration { - val LAUNCHING, LOADING, RUNNING, KILLED, FAILED, LOST = Value + val LAUNCHING, LOADING, RUNNING, KILLED, FAILED, LOST, EXITED = Value type ExecutorState = Value - def isFinished(state: ExecutorState): Boolean = Seq(KILLED, FAILED, LOST).contains(state) + def isFinished(state: ExecutorState): Boolean = Seq(KILLED, FAILED, LOST, EXITED).contains(state) } 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 f4eb1601be3e4..47dbcd87c35b5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -25,6 +25,7 @@ import scala.collection.mutable.ListBuffer import scala.concurrent.{Await, future, promise} import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ +import scala.language.postfixOps import scala.sys.process._ import org.json4s._ diff --git a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala new file mode 100644 index 0000000000000..0d6751f3fa6d2 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.net.URI + +import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConversions._ + +import org.apache.spark.api.python.{PythonUtils, RedirectThread} +import org.apache.spark.util.Utils + +/** + * A main class used by spark-submit to launch Python applications. It executes python as a + * subprocess and then has it connect back to the JVM to access system properties, etc. + */ +object PythonRunner { + def main(args: Array[String]) { + val pythonFile = args(0) + val pyFiles = args(1) + val otherArgs = args.slice(2, args.length) + val pythonExec = sys.env.get("PYSPARK_PYTHON").getOrElse("python") // TODO: get this from conf + + // Format python file paths before adding them to the PYTHONPATH + val formattedPythonFile = formatPath(pythonFile) + val formattedPyFiles = formatPaths(pyFiles) + + // Launch a Py4J gateway server for the process to connect to; this will let it see our + // Java system properties and such + val gatewayServer = new py4j.GatewayServer(null, 0) + gatewayServer.start() + + // Build up a PYTHONPATH that includes the Spark assembly JAR (where this class is), the + // python directories in SPARK_HOME (if set), and any files in the pyFiles argument + val pathElements = new ArrayBuffer[String] + pathElements ++= formattedPyFiles + pathElements += PythonUtils.sparkPythonPath + pathElements += sys.env.getOrElse("PYTHONPATH", "") + val pythonPath = PythonUtils.mergePythonPaths(pathElements: _*) + + // Launch Python process + val builder = new ProcessBuilder(Seq(pythonExec, "-u", formattedPythonFile) ++ otherArgs) + val env = builder.environment() + env.put("PYTHONPATH", pythonPath) + env.put("PYSPARK_GATEWAY_PORT", "" + gatewayServer.getListeningPort) + builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize + val process = builder.start() + + new RedirectThread(process.getInputStream, System.out, "redirect output").start() + + System.exit(process.waitFor()) + } + + /** + * Format the python file path so that it can be added to the PYTHONPATH correctly. + * + * Python does not understand URI schemes in paths. Before adding python files to the + * PYTHONPATH, we need to extract the path from the URI. This is safe to do because we + * currently only support local python files. + */ + def formatPath(path: String, testWindows: Boolean = false): String = { + if (Utils.nonLocalPaths(path, testWindows).nonEmpty) { + throw new IllegalArgumentException("Launching Python applications through " + + s"spark-submit is currently only supported for local files: $path") + } + val windows = Utils.isWindows || testWindows + var formattedPath = if (windows) Utils.formatWindowsPath(path) else path + + // Strip the URI scheme from the path + formattedPath = + new URI(formattedPath).getScheme match { + case Utils.windowsDrive(d) if windows => formattedPath + case null => formattedPath + case _ => new URI(formattedPath).getPath + } + + // Guard against malformed paths potentially throwing NPE + if (formattedPath == null) { + throw new IllegalArgumentException(s"Python file path is malformed: $path") + } + + // In Windows, the drive should not be prefixed with "/" + // For instance, python does not understand "/C:/path/to/sheep.py" + formattedPath = if (windows) formattedPath.stripPrefix("/") else formattedPath + formattedPath + } + + /** + * Format each python file path in the comma-delimited list of paths, so it can be + * added to the PYTHONPATH correctly. + */ + def formatPaths(paths: String, testWindows: Boolean = false): Array[String] = { + Option(paths).getOrElse("") + .split(",") + .filter(_.nonEmpty) + .map { p => formatPath(p, testWindows) } + } + +} 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 9bdbfb33bf54f..148115d3ed351 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -24,25 +24,36 @@ import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation -import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.{Logging, SparkContext, SparkException} import scala.collection.JavaConversions._ /** * Contains util methods to interact with Hadoop from Spark. */ -class SparkHadoopUtil { +class SparkHadoopUtil extends Logging { val conf: Configuration = newConfiguration() UserGroupInformation.setConfiguration(conf) - def runAsUser(user: String)(func: () => Unit) { + /** + * Runs the given function with a Hadoop UserGroupInformation as a thread local variable + * (distributed to child threads), used for authenticating HDFS and YARN calls. + * + * IMPORTANT NOTE: If this function is going to be called repeated in the same process + * you need to look https://issues.apache.org/jira/browse/HDFS-3545 and possibly + * do a FileSystem.closeAllForUGI in order to avoid leaking Filesystems + */ + def runAsSparkUser(func: () => Unit) { + val user = Option(System.getenv("SPARK_USER")).getOrElse(SparkContext.SPARK_UNKNOWN_USER) if (user != SparkContext.SPARK_UNKNOWN_USER) { + logDebug("running as user: " + user) val ugi = UserGroupInformation.createRemoteUser(user) transferCredentials(UserGroupInformation.getCurrentUser(), ugi) ugi.doAs(new PrivilegedExceptionAction[Unit] { def run: Unit = func() }) } else { + logDebug("running as SPARK_UNKNOWN_USER") func() } } @@ -75,6 +86,10 @@ class SparkHadoopUtil { def getSecretKeyFromUserCredentials(key: String): Array[Byte] = { null } + def loginUserFromKeytab(principalName: String, keytabFilename: String) { + UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename) + } + } object SparkHadoopUtil { @@ -88,7 +103,7 @@ object SparkHadoopUtil { .newInstance() .asInstanceOf[SparkHadoopUtil] } catch { - case th: Throwable => throw new SparkException("Unable to load YARN support", th) + case e: Exception => throw new SparkException("Unable to load YARN support", e) } } else { new SparkHadoopUtil 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 e05fbfe321495..7e9a9344e61f9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -17,14 +17,14 @@ package org.apache.spark.deploy -import java.io.{PrintStream, File} -import java.net.URL +import java.io.{File, PrintStream} +import java.lang.reflect.InvocationTargetException +import java.net.{URI, URL} -import org.apache.spark.executor.ExecutorURLClassLoader +import scala.collection.mutable.{ArrayBuffer, HashMap, Map} -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.Map +import org.apache.spark.executor.ExecutorURLClassLoader +import org.apache.spark.util.Utils /** * Scala code behind the spark-submit script. The script handles setting up the classpath with @@ -40,6 +40,12 @@ object SparkSubmit { private var clusterManager: Int = LOCAL + /** + * Special primary resource names that represent shells rather than application jars. + */ + private val SPARK_SHELL = "spark-shell" + private val PYSPARK_SHELL = "pyspark-shell" + def main(args: Array[String]) { val appArgs = new SparkSubmitArguments(args) if (appArgs.verbose) { @@ -54,148 +60,218 @@ object SparkSubmit { private[spark] var exitFn: () => Unit = () => System.exit(-1) private[spark] def printErrorAndExit(str: String) = { - printStream.println("error: " + str) - printStream.println("run with --help for more information or --verbose for debugging output") + printStream.println("Error: " + str) + printStream.println("Run with --help for usage help or --verbose for debug output") exitFn() } - private[spark] def printWarning(str: String) = printStream.println("warning: " + str) + private[spark] def printWarning(str: String) = printStream.println("Warning: " + str) /** - * @return - * a tuple containing the arguments for the child, a list of classpath - * entries for the child, and the main class for the child + * @return a tuple containing the arguments for the child, a list of classpath + * entries for the child, a list of system properties, a list of env vars + * and the main class for the child */ - private[spark] def createLaunchEnv(appArgs: SparkSubmitArguments): (ArrayBuffer[String], - ArrayBuffer[String], Map[String, String], String) = { - if (appArgs.master.startsWith("local")) { + private[spark] def createLaunchEnv(args: SparkSubmitArguments) + : (ArrayBuffer[String], ArrayBuffer[String], Map[String, String], String) = { + if (args.master.startsWith("local")) { clusterManager = LOCAL - } else if (appArgs.master.startsWith("yarn")) { + } else if (args.master.startsWith("yarn")) { clusterManager = YARN - } else if (appArgs.master.startsWith("spark")) { + } else if (args.master.startsWith("spark")) { clusterManager = STANDALONE - } else if (appArgs.master.startsWith("mesos")) { + } else if (args.master.startsWith("mesos")) { clusterManager = MESOS } else { - printErrorAndExit("master must start with yarn, mesos, spark, or local") + printErrorAndExit("Master must start with yarn, mesos, spark, or local") } // Because "yarn-cluster" and "yarn-client" encapsulate both the master // and deploy mode, we have some logic to infer the master and deploy mode // from each other if only one is specified, or exit early if they are at odds. - if (appArgs.deployMode == null && - (appArgs.master == "yarn-standalone" || appArgs.master == "yarn-cluster")) { - appArgs.deployMode = "cluster" + if (args.deployMode == null && + (args.master == "yarn-standalone" || args.master == "yarn-cluster")) { + args.deployMode = "cluster" } - if (appArgs.deployMode == "cluster" && appArgs.master == "yarn-client") { + if (args.deployMode == "cluster" && args.master == "yarn-client") { printErrorAndExit("Deploy mode \"cluster\" and master \"yarn-client\" are not compatible") } - if (appArgs.deployMode == "client" && - (appArgs.master == "yarn-standalone" || appArgs.master == "yarn-cluster")) { - printErrorAndExit("Deploy mode \"client\" and master \"" + appArgs.master + if (args.deployMode == "client" && + (args.master == "yarn-standalone" || args.master == "yarn-cluster")) { + printErrorAndExit("Deploy mode \"client\" and master \"" + args.master + "\" are not compatible") } - if (appArgs.deployMode == "cluster" && appArgs.master.startsWith("yarn")) { - appArgs.master = "yarn-cluster" + if (args.deployMode == "cluster" && args.master.startsWith("yarn")) { + args.master = "yarn-cluster" } - if (appArgs.deployMode != "cluster" && appArgs.master.startsWith("yarn")) { - appArgs.master = "yarn-client" + if (args.deployMode != "cluster" && args.master.startsWith("yarn")) { + args.master = "yarn-client" } - val deployOnCluster = Option(appArgs.deployMode).getOrElse("client") == "cluster" + val deployOnCluster = Option(args.deployMode).getOrElse("client") == "cluster" val childClasspath = new ArrayBuffer[String]() val childArgs = new ArrayBuffer[String]() val sysProps = new HashMap[String, String]() var childMainClass = "" + val isPython = args.isPython + val isYarnCluster = clusterManager == YARN && deployOnCluster + if (clusterManager == MESOS && deployOnCluster) { - printErrorAndExit("Mesos does not support running the driver on the cluster") + printErrorAndExit("Cannot currently run driver on the cluster in Mesos") } + // If we're running a python app, set the main class to our specific python runner + if (isPython) { + if (deployOnCluster) { + printErrorAndExit("Cannot currently run Python driver programs on cluster") + } + if (args.primaryResource == PYSPARK_SHELL) { + args.mainClass = "py4j.GatewayServer" + args.childArgs = ArrayBuffer("--die-on-broken-pipe", "0") + } else { + // If a python file is provided, add it to the child arguments and list of files to deploy. + // Usage: PythonAppRunner
[app arguments] + args.mainClass = "org.apache.spark.deploy.PythonRunner" + args.childArgs = ArrayBuffer(args.primaryResource, args.pyFiles) ++ args.childArgs + args.files = mergeFileLists(args.files, args.primaryResource) + } + args.files = mergeFileLists(args.files, args.pyFiles) + // Format python file paths properly before adding them to the PYTHONPATH + sysProps("spark.submit.pyFiles") = PythonRunner.formatPaths(args.pyFiles).mkString(",") + } + + // If we're deploying into YARN, use yarn.Client as a wrapper around the user class if (!deployOnCluster) { - childMainClass = appArgs.mainClass - childClasspath += appArgs.primaryResource + childMainClass = args.mainClass + if (isUserJar(args.primaryResource)) { + childClasspath += args.primaryResource + } } else if (clusterManager == YARN) { childMainClass = "org.apache.spark.deploy.yarn.Client" - childArgs += ("--jar", appArgs.primaryResource) - childArgs += ("--class", appArgs.mainClass) + childArgs += ("--jar", args.primaryResource) + childArgs += ("--class", args.mainClass) } + // Make sure YARN is included in our build if we're trying to use it + if (clusterManager == YARN) { + if (!Utils.classIsLoadable("org.apache.spark.deploy.yarn.Client") && !Utils.isTesting) { + printErrorAndExit("Could not load YARN classes. " + + "This copy of Spark may not have been compiled with YARN support.") + } + } + + // Special flag to avoid deprecation warnings at the client + sysProps("SPARK_SUBMIT") = "true" + + // A list of rules to map each argument to system properties or command-line options in + // each deploy mode; we iterate through these below val options = List[OptionAssigner]( - new OptionAssigner(appArgs.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), - new OptionAssigner(appArgs.driverMemory, YARN, true, clOption = "--driver-memory"), - new OptionAssigner(appArgs.name, YARN, true, clOption = "--name"), - new OptionAssigner(appArgs.queue, YARN, true, clOption = "--queue"), - new OptionAssigner(appArgs.queue, YARN, false, sysProp = "spark.yarn.queue"), - new OptionAssigner(appArgs.numExecutors, YARN, true, clOption = "--num-executors"), - new OptionAssigner(appArgs.numExecutors, YARN, false, sysProp = "spark.executor.instances"), - new OptionAssigner(appArgs.executorMemory, YARN, true, clOption = "--executor-memory"), - new OptionAssigner(appArgs.executorMemory, STANDALONE | MESOS | YARN, false, + OptionAssigner(args.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), + OptionAssigner(args.name, ALL_CLUSTER_MGRS, false, sysProp = "spark.app.name"), + OptionAssigner(args.name, YARN, true, clOption = "--name", sysProp = "spark.app.name"), + OptionAssigner(args.driverExtraClassPath, STANDALONE | YARN, true, + sysProp = "spark.driver.extraClassPath"), + OptionAssigner(args.driverExtraJavaOptions, STANDALONE | YARN, true, + sysProp = "spark.driver.extraJavaOptions"), + OptionAssigner(args.driverExtraLibraryPath, STANDALONE | YARN, true, + sysProp = "spark.driver.extraLibraryPath"), + OptionAssigner(args.driverMemory, YARN, true, clOption = "--driver-memory"), + OptionAssigner(args.driverMemory, STANDALONE, true, clOption = "--memory"), + OptionAssigner(args.driverCores, STANDALONE, true, clOption = "--cores"), + OptionAssigner(args.queue, YARN, true, clOption = "--queue"), + OptionAssigner(args.queue, YARN, false, sysProp = "spark.yarn.queue"), + OptionAssigner(args.numExecutors, YARN, true, clOption = "--num-executors"), + OptionAssigner(args.numExecutors, YARN, false, sysProp = "spark.executor.instances"), + OptionAssigner(args.executorMemory, YARN, true, clOption = "--executor-memory"), + OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, false, sysProp = "spark.executor.memory"), - new OptionAssigner(appArgs.driverMemory, STANDALONE, true, clOption = "--memory"), - new OptionAssigner(appArgs.driverCores, STANDALONE, true, clOption = "--cores"), - new OptionAssigner(appArgs.executorCores, YARN, true, clOption = "--executor-cores"), - new OptionAssigner(appArgs.executorCores, YARN, false, sysProp = "spark.executor.cores"), - new OptionAssigner(appArgs.totalExecutorCores, STANDALONE | MESOS, false, + OptionAssigner(args.executorCores, YARN, true, clOption = "--executor-cores"), + OptionAssigner(args.executorCores, YARN, false, sysProp = "spark.executor.cores"), + OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, false, sysProp = "spark.cores.max"), - new OptionAssigner(appArgs.files, YARN, false, sysProp = "spark.yarn.dist.files"), - new OptionAssigner(appArgs.files, YARN, true, clOption = "--files"), - new OptionAssigner(appArgs.archives, YARN, false, sysProp = "spark.yarn.dist.archives"), - new OptionAssigner(appArgs.archives, YARN, true, clOption = "--archives"), - new OptionAssigner(appArgs.jars, YARN, true, clOption = "--addJars") + OptionAssigner(args.files, YARN, false, sysProp = "spark.yarn.dist.files"), + OptionAssigner(args.files, YARN, true, clOption = "--files"), + OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.files"), + OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, true, sysProp = "spark.files"), + OptionAssigner(args.archives, YARN, false, sysProp = "spark.yarn.dist.archives"), + OptionAssigner(args.archives, YARN, true, clOption = "--archives"), + OptionAssigner(args.jars, YARN, true, clOption = "--addJars"), + OptionAssigner(args.jars, ALL_CLUSTER_MGRS, false, sysProp = "spark.jars") ) - // more jars - if (appArgs.jars != null && !deployOnCluster) { - for (jar <- appArgs.jars.split(",")) { + // For client mode make any added jars immediately visible on the classpath + if (args.jars != null && !deployOnCluster) { + for (jar <- args.jars.split(",")) { childClasspath += jar } } + // Map all arguments to command-line options or system properties for our chosen mode for (opt <- options) { if (opt.value != null && deployOnCluster == opt.deployOnCluster && - (clusterManager & opt.clusterManager) != 0) { + (clusterManager & opt.clusterManager) != 0) { if (opt.clOption != null) { childArgs += (opt.clOption, opt.value) - } else if (opt.sysProp != null) { + } + if (opt.sysProp != null) { sysProps.put(opt.sysProp, opt.value) } } } + // 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 && !isPython) { + var jars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq()) + if (isUserJar(args.primaryResource)) { + jars = jars ++ Seq(args.primaryResource) + } + sysProps.put("spark.jars", jars.mkString(",")) + } + + // Standalone cluster specific configurations if (deployOnCluster && clusterManager == STANDALONE) { - if (appArgs.supervise) { + if (args.supervise) { childArgs += "--supervise" } - childMainClass = "org.apache.spark.deploy.Client" childArgs += "launch" - childArgs += (appArgs.master, appArgs.primaryResource, appArgs.mainClass) + childArgs += (args.master, args.primaryResource, args.mainClass) } - // args - if (appArgs.childArgs != null) { + // Arguments to be passed to user program + if (args.childArgs != null) { if (!deployOnCluster || clusterManager == STANDALONE) { - childArgs ++= appArgs.childArgs + childArgs ++= args.childArgs } else if (clusterManager == YARN) { - for (arg <- appArgs.childArgs) { - childArgs += ("--args", arg) + for (arg <- args.childArgs) { + childArgs += ("--arg", arg) } } } + // Read from default spark properties, if any + for ((k, v) <- args.getDefaultSparkProperties) { + if (!sysProps.contains(k)) sysProps(k) = v + } + (childArgs, childClasspath, sysProps, childMainClass) } - private def launch(childArgs: ArrayBuffer[String], childClasspath: ArrayBuffer[String], - sysProps: Map[String, String], childMainClass: String, verbose: Boolean = false) { - + private def launch( + childArgs: ArrayBuffer[String], + childClasspath: ArrayBuffer[String], + sysProps: Map[String, String], + childMainClass: String, + verbose: Boolean = false) { if (verbose) { - System.err.println(s"Main class:\n$childMainClass") - System.err.println(s"Arguments:\n${childArgs.mkString("\n")}") - System.err.println(s"System properties:\n${sysProps.mkString("\n")}") - System.err.println(s"Classpath elements:\n${childClasspath.mkString("\n")}") - System.err.println("\n") + printStream.println(s"Main class:\n$childMainClass") + printStream.println(s"Arguments:\n${childArgs.mkString("\n")}") + printStream.println(s"System properties:\n${sysProps.mkString("\n")}") + printStream.println(s"Classpath elements:\n${childClasspath.mkString("\n")}") + printStream.println("\n") } val loader = new ExecutorURLClassLoader(new Array[URL](0), @@ -212,23 +288,71 @@ object SparkSubmit { val mainClass = Class.forName(childMainClass, true, loader) val mainMethod = mainClass.getMethod("main", new Array[String](0).getClass) - mainMethod.invoke(null, childArgs.toArray) + try { + mainMethod.invoke(null, childArgs.toArray) + } catch { + case e: InvocationTargetException => e.getCause match { + case cause: Throwable => throw cause + case null => throw e + } + } } private def addJarToClasspath(localJar: String, loader: ExecutorURLClassLoader) { - val localJarFile = new File(localJar) - if (!localJarFile.exists()) { - printWarning(s"Jar $localJar does not exist, skipping.") + val uri = Utils.resolveURI(localJar) + uri.getScheme match { + case "file" | "local" => + val file = new File(uri.getPath) + if (file.exists()) { + loader.addURL(file.toURI.toURL) + } else { + printWarning(s"Local jar $file does not exist, skipping.") + } + case _ => + printWarning(s"Skip remote jar $uri.") } + } - val url = localJarFile.getAbsoluteFile.toURI.toURL - loader.addURL(url) + /** + * Return whether the given primary resource represents a user jar. + */ + private def isUserJar(primaryResource: String): Boolean = { + !isShell(primaryResource) && !isPython(primaryResource) + } + + /** + * Return whether the given primary resource represents a shell. + */ + private[spark] def isShell(primaryResource: String): Boolean = { + primaryResource == SPARK_SHELL || primaryResource == PYSPARK_SHELL + } + + /** + * Return whether the given primary resource requires running python. + */ + private[spark] def isPython(primaryResource: String): Boolean = { + primaryResource.endsWith(".py") || primaryResource == PYSPARK_SHELL + } + + /** + * Merge a sequence of comma-separated file lists, some of which may be null to indicate + * no files, into a single comma-separated string. + */ + private[spark] def mergeFileLists(lists: String*): String = { + val merged = lists.filter(_ != null) + .flatMap(_.split(",")) + .mkString(",") + if (merged == "") null else merged } } -private[spark] class OptionAssigner(val value: String, - val clusterManager: Int, - val deployOnCluster: Boolean, - val clOption: String = null, - val sysProp: String = null -) { } +/** + * Provides an indirection layer for passing arguments as system properties or flags to + * the user's driver program or to downstream launcher tools. + */ +private[spark] case class OptionAssigner( + value: String, + clusterManager: Int, + deployOnCluster: Boolean, + clOption: String = null, + sysProp: String = null) 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 834b3df2f164b..f1032ea8dbada 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -17,18 +17,30 @@ package org.apache.spark.deploy -import scala.collection.mutable.ArrayBuffer +import java.io.{File, FileInputStream, IOException} +import java.util.Properties +import java.util.jar.JarFile + +import scala.collection.JavaConversions._ +import scala.collection.mutable.{ArrayBuffer, HashMap} + +import org.apache.spark.SparkException +import org.apache.spark.util.Utils /** * Parses and encapsulates arguments from the spark-submit script. */ -private[spark] class SparkSubmitArguments(args: Array[String]) { - var master: String = "local" +private[spark] class SparkSubmitArguments(args: Seq[String]) { + var master: String = null var deployMode: String = null var executorMemory: String = null var executorCores: String = null var totalExecutorCores: String = null + var propertiesFile: String = null var driverMemory: String = null + var driverExtraClassPath: String = null + var driverExtraLibraryPath: String = null + var driverExtraJavaOptions: String = null var driverCores: String = null var supervise: Boolean = false var queue: String = null @@ -41,128 +53,280 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { var childArgs: ArrayBuffer[String] = new ArrayBuffer[String]() var jars: String = null var verbose: Boolean = false + var isPython: Boolean = false + var pyFiles: String = null - loadEnvVars() parseOpts(args.toList) + loadDefaults() + checkRequiredArguments() + + /** Return default present in the currently defined defaults file. */ + def getDefaultSparkProperties = { + val defaultProperties = new HashMap[String, String]() + if (verbose) SparkSubmit.printStream.println(s"Using properties file: $propertiesFile") + Option(propertiesFile).foreach { filename => + val file = new File(filename) + SparkSubmitArguments.getPropertiesFromFile(file).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 + } + + /** Fill in any undefined values based on the current properties file or built-in defaults. */ + private def loadDefaults(): Unit = { + + // Use common defaults file, if not specified by user + if (propertiesFile == null) { + sys.env.get("SPARK_HOME").foreach { sparkHome => + val sep = File.separator + val defaultPath = s"${sparkHome}${sep}conf${sep}spark-defaults.conf" + val file = new File(defaultPath) + if (file.exists()) { + propertiesFile = file.getAbsolutePath + } + } + } + + val defaultProperties = getDefaultSparkProperties + // Use properties file as fallback for values which have a direct analog to + // arguments in this script. + master = Option(master).getOrElse(defaultProperties.get("spark.master").orNull) + executorMemory = Option(executorMemory) + .getOrElse(defaultProperties.get("spark.executor.memory").orNull) + executorCores = Option(executorCores) + .getOrElse(defaultProperties.get("spark.executor.cores").orNull) + totalExecutorCores = Option(totalExecutorCores) + .getOrElse(defaultProperties.get("spark.cores.max").orNull) + name = Option(name).getOrElse(defaultProperties.get("spark.app.name").orNull) + jars = Option(jars).getOrElse(defaultProperties.get("spark.jars").orNull) + + // This supports env vars in older versions of Spark + master = Option(master).getOrElse(System.getenv("MASTER")) + deployMode = Option(deployMode).getOrElse(System.getenv("DEPLOY_MODE")) + + // Try to set main class from JAR if no --class argument is given + if (mainClass == null && !isPython && primaryResource != null) { + try { + val jar = new JarFile(primaryResource) + // Note that this might still return null if no main-class is set; we catch that later + mainClass = jar.getManifest.getMainAttributes.getValue("Main-Class") + } catch { + case e: Exception => + SparkSubmit.printErrorAndExit("Cannot load main class from JAR: " + primaryResource) + return + } + } + + // Global defaults. These should be keep to minimum to avoid confusing behavior. + master = Option(master).getOrElse("local[*]") + + // Set name from main class if not given + name = Option(name).orElse(Option(mainClass)).orNull + if (name == null && primaryResource != null) { + name = Utils.stripDirectory(primaryResource) + } + } + + /** Ensure that required fields exists. Call this only once all defaults are loaded. */ + private def checkRequiredArguments() = { + if (args.length == 0) { + printUsageAndExit(-1) + } + if (primaryResource == null) { + SparkSubmit.printErrorAndExit("Must specify a primary resource (JAR or Python file)") + } + if (mainClass == null && !isPython) { + SparkSubmit.printErrorAndExit("No main class set in JAR; please specify one with --class") + } + if (pyFiles != null && !isPython) { + SparkSubmit.printErrorAndExit("--py-files given but primary resource is not a Python script") + } - // Sanity checks - if (args.length == 0) printUsageAndExit(-1) - if (primaryResource == null) SparkSubmit.printErrorAndExit("Must specify a primary resource") - if (mainClass == null) SparkSubmit.printErrorAndExit("Must specify a main class with --class") + // Require all python files to be local, so we can add them to the PYTHONPATH + if (isPython) { + if (Utils.nonLocalPaths(primaryResource).nonEmpty) { + SparkSubmit.printErrorAndExit(s"Only local python files are supported: $primaryResource") + } + val nonLocalPyFiles = Utils.nonLocalPaths(pyFiles).mkString(",") + if (nonLocalPyFiles.nonEmpty) { + SparkSubmit.printErrorAndExit( + s"Only local additional python files are supported: $nonLocalPyFiles") + } + } + + if (master.startsWith("yarn")) { + val hasHadoopEnv = sys.env.contains("HADOOP_CONF_DIR") || sys.env.contains("YARN_CONF_DIR") + if (!hasHadoopEnv && !Utils.isTesting) { + throw new Exception(s"When running with master '$master' " + + "either HADOOP_CONF_DIR or YARN_CONF_DIR must be set in the environment.") + } + } + } override def toString = { s"""Parsed arguments: - | master $master - | deployMode $deployMode - | executorMemory $executorMemory - | executorCores $executorCores - | totalExecutorCores $totalExecutorCores - | driverMemory $driverMemory - | drivercores $driverCores - | supervise $supervise - | queue $queue - | numExecutors $numExecutors - | files $files - | archives $archives - | mainClass $mainClass - | primaryResource $primaryResource - | name $name - | childArgs [${childArgs.mkString(" ")}] - | jars $jars - | verbose $verbose + | master $master + | deployMode $deployMode + | executorMemory $executorMemory + | executorCores $executorCores + | totalExecutorCores $totalExecutorCores + | propertiesFile $propertiesFile + | driverMemory $driverMemory + | driverCores $driverCores + | driverExtraClassPath $driverExtraClassPath + | driverExtraLibraryPath $driverExtraLibraryPath + | driverExtraJavaOptions $driverExtraJavaOptions + | supervise $supervise + | queue $queue + | numExecutors $numExecutors + | files $files + | pyFiles $pyFiles + | archives $archives + | mainClass $mainClass + | primaryResource $primaryResource + | name $name + | childArgs [${childArgs.mkString(" ")}] + | jars $jars + | verbose $verbose + | + |Default properties from $propertiesFile: + |${getDefaultSparkProperties.mkString(" ", "\n ", "\n")} """.stripMargin } - private def loadEnvVars() { - Option(System.getenv("MASTER")).map(master = _) - Option(System.getenv("DEPLOY_MODE")).map(deployMode = _) - } + /** Fill in values by parsing user options. */ + private def parseOpts(opts: Seq[String]): Unit = { + // Delineates parsing of Spark options from parsing of user options. + var inSparkOpts = true + parse(opts) - private def parseOpts(opts: List[String]): Unit = opts match { - case ("--name") :: value :: tail => - name = value - parseOpts(tail) + def parse(opts: Seq[String]): Unit = opts match { + case ("--name") :: value :: tail => + name = value + parse(tail) - case ("--master") :: value :: tail => - master = value - parseOpts(tail) + case ("--master") :: value :: tail => + master = value + parse(tail) - case ("--class") :: value :: tail => - mainClass = value - parseOpts(tail) + case ("--class") :: value :: tail => + mainClass = value + parse(tail) - case ("--deploy-mode") :: value :: tail => - if (value != "client" && value != "cluster") { - SparkSubmit.printErrorAndExit("--deploy-mode must be either \"client\" or \"cluster\"") - } - deployMode = value - parseOpts(tail) + case ("--deploy-mode") :: value :: tail => + if (value != "client" && value != "cluster") { + SparkSubmit.printErrorAndExit("--deploy-mode must be either \"client\" or \"cluster\"") + } + deployMode = value + parse(tail) - case ("--num-executors") :: value :: tail => - numExecutors = value - parseOpts(tail) + case ("--num-executors") :: value :: tail => + numExecutors = value + parse(tail) - case ("--total-executor-cores") :: value :: tail => - totalExecutorCores = value - parseOpts(tail) + case ("--total-executor-cores") :: value :: tail => + totalExecutorCores = value + parse(tail) - case ("--executor-cores") :: value :: tail => - executorCores = value - parseOpts(tail) + case ("--executor-cores") :: value :: tail => + executorCores = value + parse(tail) - case ("--executor-memory") :: value :: tail => - executorMemory = value - parseOpts(tail) + case ("--executor-memory") :: value :: tail => + executorMemory = value + parse(tail) - case ("--driver-memory") :: value :: tail => - driverMemory = value - parseOpts(tail) + case ("--driver-memory") :: value :: tail => + driverMemory = value + parse(tail) - case ("--driver-cores") :: value :: tail => - driverCores = value - parseOpts(tail) + case ("--driver-cores") :: value :: tail => + driverCores = value + parse(tail) - case ("--supervise") :: tail => - supervise = true - parseOpts(tail) + case ("--driver-class-path") :: value :: tail => + driverExtraClassPath = value + parse(tail) - case ("--queue") :: value :: tail => - queue = value - parseOpts(tail) + case ("--driver-java-options") :: value :: tail => + driverExtraJavaOptions = value + parse(tail) - case ("--files") :: value :: tail => - files = value - parseOpts(tail) + case ("--driver-library-path") :: value :: tail => + driverExtraLibraryPath = value + parse(tail) - case ("--archives") :: value :: tail => - archives = value - parseOpts(tail) + case ("--properties-file") :: value :: tail => + propertiesFile = value + parse(tail) - case ("--arg") :: value :: tail => - childArgs += value - parseOpts(tail) + case ("--supervise") :: tail => + supervise = true + parse(tail) - case ("--jars") :: value :: tail => - jars = value - parseOpts(tail) + case ("--queue") :: value :: tail => + queue = value + parse(tail) - case ("--help" | "-h") :: tail => - printUsageAndExit(0) + case ("--files") :: value :: tail => + files = Utils.resolveURIs(value) + parse(tail) - case ("--verbose" | "-v") :: tail => - verbose = true - parseOpts(tail) + case ("--py-files") :: value :: tail => + pyFiles = Utils.resolveURIs(value) + parse(tail) - case value :: tail => - if (primaryResource != null) { - val error = s"Found two conflicting resources, $value and $primaryResource." + - " Expecting only one resource." - SparkSubmit.printErrorAndExit(error) - } - primaryResource = value - parseOpts(tail) + case ("--archives") :: value :: tail => + archives = Utils.resolveURIs(value) + parse(tail) + + case ("--jars") :: value :: tail => + jars = Utils.resolveURIs(value) + parse(tail) + + case ("--help" | "-h") :: tail => + printUsageAndExit(0) - case Nil => + case ("--verbose" | "-v") :: tail => + verbose = true + parse(tail) + + case value :: tail => + if (inSparkOpts) { + value match { + // convert --foo=bar to --foo bar + case v if v.startsWith("--") && v.contains("=") && v.split("=").size == 2 => + val parts = v.split("=") + parse(Seq(parts(0), parts(1)) ++ tail) + case v if v.startsWith("-") => + val errMessage = s"Unrecognized option '$value'." + SparkSubmit.printErrorAndExit(errMessage) + case v => + primaryResource = + if (!SparkSubmit.isShell(v)) { + Utils.resolveURI(v).toString + } else { + v + } + inSparkOpts = false + isPython = SparkSubmit.isPython(v) + parse(tail) + } + } else { + if (!value.isEmpty) { + childArgs += value + } + parse(tail) + } + + case Nil => + } } private def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { @@ -171,18 +335,33 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { outStream.println("Unknown/unsupported param " + unknownParam) } outStream.println( - """Usage: spark-submit [options] + """Usage: spark-submit [options] [app options] |Options: | --master MASTER_URL spark://host:port, mesos://host:port, yarn, or local. - | --deploy-mode DEPLOY_MODE Mode to deploy the app in, either 'client' or 'cluster'. - | --class CLASS_NAME Name of your app's main class (required for Java apps). - | --arg ARG Argument to be passed to your application's main class. This - | option can be specified multiple times for multiple args. + | --deploy-mode DEPLOY_MODE Where to run the driver program: either "client" to run + | on the local machine, or "cluster" to run inside cluster. + | --class CLASS_NAME Your application's main class (for Java / Scala apps). + | --name NAME A name of your application. + | --jars JARS Comma-separated list of local jars to include on the driver + | and executor classpaths. + | --py-files PY_FILES Comma-separated list of .zip, .egg, or .py files to place + | on the PYTHONPATH for Python apps. + | --files FILES Comma-separated list of files to be placed in the working + | directory of each executor. + | --properties-file FILE Path to a file from which to load extra properties. If not + | specified, this will look for conf/spark-defaults.conf. + | | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512M). - | --name NAME The name of your application (Default: 'Spark'). - | --jars JARS A comma-separated list of local jars to include on the - | driver classpath and that SparkContext.addJar will work - | with. Doesn't work on standalone with 'cluster' deploy mode. + | --driver-java-options Extra Java options to pass to the driver. + | --driver-library-path Extra library path entries to pass to the driver. + | --driver-class-path Extra class path entries to pass to the driver. Note that + | jars added with --jars are automatically included in the + | classpath. + | + | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G). + | + | --help, -h Show this help message and exit + | --verbose, -v Print additional debug output | | Spark standalone with cluster deploy mode only: | --driver-cores NUM Cores for driver (Default: 1). @@ -193,14 +372,31 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { | | YARN-only: | --executor-cores NUM Number of cores per executor (Default: 1). - | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G). - | --queue QUEUE_NAME The YARN queue to submit to (Default: 'default'). - | --num-executors NUM Number of executors to (Default: 2). - | --files FILES Comma separated list of files to be placed in the working dir - | of each executor. + | --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 dir of each executor.""".stripMargin + | working directory of each executor.""".stripMargin ) SparkSubmit.exitFn() } } + +object SparkSubmitArguments { + /** Load properties present in the given file. */ + def getPropertiesFromFile(file: File): Seq[(String, String)] = { + require(file.exists(), s"Properties file $file does not exist") + require(file.isFile(), s"Properties file $file is not a normal file") + val inputStream = new FileInputStream(file) + try { + val properties = new Properties() + properties.load(inputStream) + properties.stringPropertyNames().toSeq.map(k => (k, properties(k).trim)) + } catch { + case e: IOException => + val message = s"Failed when loading Spark properties file $file" + throw new SparkException(message, e) + } finally { + inputStream.close() + } + } +} 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 8901806de9262..d38e9e79204c2 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, SparkException} 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.AkkaUtils +import org.apache.spark.util.{Utils, AkkaUtils} /** * Interface allowing applications to speak with a Spark deploy cluster. Takes a master URL, @@ -60,6 +60,7 @@ private[spark] class AppClient( var master: ActorSelection = null var alreadyDisconnected = false // To avoid calling listener.disconnected() multiple times var alreadyDead = false // To avoid calling listener.dead() multiple times + var registrationRetryTimer: Option[Cancellable] = None override def preStart() { context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) @@ -83,22 +84,22 @@ private[spark] class AppClient( def registerWithMaster() { tryRegisterAllMasters() - import context.dispatcher var retries = 0 - lazy val retryTimer: Cancellable = + registrationRetryTimer = Some { context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) { - retries += 1 - if (registered) { - retryTimer.cancel() - } else if (retries >= REGISTRATION_RETRIES) { - logError("All masters are unresponsive! Giving up.") - markDead() - } else { - tryRegisterAllMasters() + Utils.tryOrExit { + retries += 1 + if (registered) { + registrationRetryTimer.foreach(_.cancel()) + } else if (retries >= REGISTRATION_RETRIES) { + markDead("All masters are unresponsive! Giving up.") + } else { + tryRegisterAllMasters() + } } } - retryTimer // start timer + } } def changeMaster(url: String) { @@ -126,8 +127,7 @@ private[spark] class AppClient( listener.connected(appId) case ApplicationRemoved(message) => - logError("Master removed our application: %s; stopping client".format(message)) - markDisconnected() + markDead("Master removed our application: %s".format(message)) context.stop(self) case ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) => @@ -158,7 +158,7 @@ private[spark] class AppClient( logWarning(s"Could not connect to $address: $cause") case StopAppClient => - markDead() + markDead("Application has been stopped.") sender ! true context.stop(self) } @@ -173,12 +173,17 @@ private[spark] class AppClient( } } - def markDead() { + def markDead(reason: String) { if (!alreadyDead) { - listener.dead() + listener.dead(reason) alreadyDead = true } } + + override def postStop() { + registrationRetryTimer.foreach(_.cancel()) + } + } def start() { diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala index 1f20aa3dfa39b..e584952a9ad85 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala @@ -30,8 +30,8 @@ private[spark] trait AppClientListener { /** Disconnection may be a temporary state, as we fail over to a new Master. */ def disconnected(): Unit - /** Dead means that we couldn't find any Masters to connect to, and have given up. */ - def dead(): Unit + /** An application death is an unrecoverable failure condition. */ + def dead(reason: String): Unit def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int) 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 63f166d401059..e15a87bd38fda 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 @@ -33,8 +33,8 @@ private[spark] object TestClient { System.exit(0) } - def dead() { - logInfo("Could not connect to master") + def dead(reason: String) { + logInfo("Application died with error: " + reason) System.exit(0) } @@ -49,8 +49,8 @@ private[spark] object TestClient { val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, conf = conf, securityManager = new SecurityManager(conf)) val desc = new ApplicationDescription( - "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), - Some("dummy-spark-home"), "ignored") + "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map(), Seq(), + Seq()), Some("dummy-spark-home"), "ignored") val listener = new TestListener val client = new AppClient(actorSystem, Array(url), desc, listener, new SparkConf) client.start() 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 new file mode 100644 index 0000000000000..180c853ce3096 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.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.deploy.history + +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.spark.ui.{WebUIPage, UIUtils} + +private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { + + def render(request: HttpServletRequest): Seq[Node] = { + val appRows = parent.appIdToInfo.values.toSeq.sortBy { app => -app.lastUpdated } + val appTable = UIUtils.listingTable(appHeader, appRow, appRows) + val content = +
+
+
    +
  • Event Log Location: {parent.baseLogDir}
  • +
+ { + if (parent.appIdToInfo.size > 0) { +

+ Showing {parent.appIdToInfo.size}/{parent.getNumApplications} + Completed Application{if (parent.getNumApplications > 1) "s" else ""} +

++ + appTable + } else { +

No Completed Applications Found

+ } + } +
+
+ UIUtils.basicSparkPage(content, "History Server") + } + + private val appHeader = Seq( + "App Name", + "Started", + "Completed", + "Duration", + "Spark User", + "Log Directory", + "Last Updated") + + private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { + val appName = if (info.started) info.name else info.logDirPath.getName + val uiAddress = parent.getAddress + info.ui.basePath + val startTime = if (info.started) UIUtils.formatDate(info.startTime) else "Not started" + val endTime = if (info.completed) UIUtils.formatDate(info.endTime) else "Not completed" + val difference = if (info.started && info.completed) info.endTime - info.startTime else -1L + val duration = if (difference > 0) UIUtils.formatDuration(difference) else "---" + val sparkUser = if (info.started) info.sparkUser else "Unknown user" + val logDirectory = info.logDirPath.getName + val lastUpdated = UIUtils.formatDate(info.lastUpdated) + + {appName} + {startTime} + {endTime} + {duration} + {sparkUser} + {logDirectory} + {lastUpdated} + + } +} 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 new file mode 100644 index 0000000000000..a9c11dca5678e --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -0,0 +1,307 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.history + +import scala.collection.mutable + +import org.apache.hadoop.fs.{FileStatus, Path} + +import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.scheduler._ +import org.apache.spark.ui.{WebUI, SparkUI} +import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.util.Utils + +/** + * A web server that renders SparkUIs of completed applications. + * + * For the standalone mode, MasterWebUI already achieves this functionality. Thus, the + * main use case of the HistoryServer is in other deploy modes (e.g. Yarn or Mesos). + * + * The logging directory structure is as follows: Within the given base directory, each + * application's event logs are maintained in the application's own sub-directory. This + * is the same structure as maintained in the event log write code path in + * EventLoggingListener. + * + * @param baseLogDir The base directory in which event logs are found + */ +class HistoryServer( + val baseLogDir: String, + securityManager: SecurityManager, + conf: SparkConf) + extends WebUI(securityManager, HistoryServer.WEB_UI_PORT, conf) with Logging { + + import HistoryServer._ + + private val fileSystem = Utils.getHadoopFileSystem(baseLogDir) + private val localHost = Utils.localHostName() + private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) + + // A timestamp of when the disk was last accessed to check for log updates + private var lastLogCheckTime = -1L + + // Number of completed applications found in this directory + private var numCompletedApplications = 0 + + @volatile private var stopped = false + + /** + * A background thread that periodically checks for event log updates on disk. + * + * If a log check is invoked manually in the middle of a period, this thread re-adjusts the + * time at which it performs the next log check to maintain the same period as before. + * + * TODO: Add a mechanism to update manually. + */ + private val logCheckingThread = new Thread { + override def run(): Unit = Utils.logUncaughtExceptions { + while (!stopped) { + val now = System.currentTimeMillis + if (now - lastLogCheckTime > UPDATE_INTERVAL_MS) { + checkForLogs() + Thread.sleep(UPDATE_INTERVAL_MS) + } else { + // If the user has manually checked for logs recently, wait until + // UPDATE_INTERVAL_MS after the last check time + Thread.sleep(lastLogCheckTime + UPDATE_INTERVAL_MS - now) + } + } + } + } + + // A mapping of application ID to its history information, which includes the rendered UI + val appIdToInfo = mutable.HashMap[String, ApplicationHistoryInfo]() + + initialize() + + /** + * Initialize the history server. + * + * This starts a background thread that periodically synchronizes information displayed on + * this UI with the event logs in the provided base directory. + */ + def initialize() { + attachPage(new HistoryPage(this)) + attachHandler(createStaticHandler(STATIC_RESOURCE_DIR, "/static")) + } + + /** Bind to the HTTP server behind this web interface. */ + override def bind() { + super.bind() + logCheckingThread.start() + } + + /** + * Check for any updates to event logs in the base directory. This is only effective once + * the server has been bound. + * + * If a new completed application is found, the server renders the associated SparkUI + * from the application's event logs, attaches this UI to itself, and stores metadata + * information for this application. + * + * If the logs for an existing completed application are no longer found, the server + * removes all associated information and detaches the SparkUI. + */ + def checkForLogs() = synchronized { + if (serverInfo.isDefined) { + lastLogCheckTime = System.currentTimeMillis + logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTime)) + try { + val logStatus = fileSystem.listStatus(new Path(baseLogDir)) + val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() + val logInfos = logDirs + .sortBy { dir => getModificationTime(dir) } + .map { dir => (dir, EventLoggingListener.parseLoggingInfo(dir.getPath, fileSystem)) } + .filter { case (dir, info) => info.applicationComplete } + + // Logging information for applications that should be retained + val retainedLogInfos = logInfos.takeRight(RETAINED_APPLICATIONS) + val retainedAppIds = retainedLogInfos.map { case (dir, _) => dir.getPath.getName } + + // Remove any applications that should no longer be retained + appIdToInfo.foreach { case (appId, info) => + if (!retainedAppIds.contains(appId)) { + detachSparkUI(info.ui) + appIdToInfo.remove(appId) + } + } + + // Render the application's UI if it is not already there + retainedLogInfos.foreach { case (dir, info) => + val appId = dir.getPath.getName + if (!appIdToInfo.contains(appId)) { + renderSparkUI(dir, info) + } + } + + // Track the total number of completed applications observed this round + numCompletedApplications = logInfos.size + + } catch { + case e: Exception => logError("Exception in checking for event log updates", e) + } + } else { + logWarning("Attempted to check for event log updates before binding the server.") + } + } + + /** + * Render a new SparkUI from the event logs if the associated application is completed. + * + * HistoryServer looks for a special file that indicates application completion in the given + * directory. If this file exists, the associated application is regarded to be completed, in + * which case the server proceeds to render the SparkUI. Otherwise, the server does nothing. + */ + private def renderSparkUI(logDir: FileStatus, elogInfo: EventLoggingInfo) { + val path = logDir.getPath + val appId = path.getName + val replayBus = new ReplayListenerBus(elogInfo.logPaths, fileSystem, elogInfo.compressionCodec) + val appListener = new ApplicationEventListener + replayBus.addListener(appListener) + val appConf = conf.clone() + val appSecManager = new SecurityManager(appConf) + val ui = new SparkUI(conf, appSecManager, replayBus, appId, "/history/" + appId) + + // Do not call ui.bind() to avoid creating a new server for each application + replayBus.replay() + if (appListener.applicationStarted) { + appSecManager.setUIAcls(HISTORY_UI_ACLS_ENABLED) + appSecManager.setViewAcls(appListener.sparkUser, appListener.viewAcls) + attachSparkUI(ui) + val appName = appListener.appName + val sparkUser = appListener.sparkUser + val startTime = appListener.startTime + val endTime = appListener.endTime + val lastUpdated = getModificationTime(logDir) + ui.setAppName(appName + " (completed)") + appIdToInfo(appId) = ApplicationHistoryInfo(appId, appName, startTime, endTime, + lastUpdated, sparkUser, path, ui) + } + } + + /** Stop the server and close the file system. */ + override def stop() { + super.stop() + stopped = true + fileSystem.close() + } + + /** Attach a reconstructed UI to this server. Only valid after bind(). */ + private def attachSparkUI(ui: SparkUI) { + assert(serverInfo.isDefined, "HistoryServer must be bound before attaching SparkUIs") + ui.getHandlers.foreach(attachHandler) + addFilters(ui.getHandlers, conf) + } + + /** Detach a reconstructed UI from this server. Only valid after bind(). */ + private def detachSparkUI(ui: SparkUI) { + assert(serverInfo.isDefined, "HistoryServer must be bound before detaching SparkUIs") + ui.getHandlers.foreach(detachHandler) + } + + /** Return the address of this server. */ + def getAddress: String = "http://" + publicHost + ":" + boundPort + + /** Return the number of completed applications found, whether or not the UI is rendered. */ + def getNumApplications: Int = numCompletedApplications + + /** Return when this directory was last modified. */ + private def getModificationTime(dir: FileStatus): Long = { + try { + val logFiles = fileSystem.listStatus(dir.getPath) + if (logFiles != null && !logFiles.isEmpty) { + logFiles.map(_.getModificationTime).max + } else { + dir.getModificationTime + } + } catch { + case e: Exception => + logError("Exception in accessing modification time of %s".format(dir.getPath), e) + -1L + } + } +} + +/** + * The recommended way of starting and stopping a HistoryServer is through the scripts + * start-history-server.sh and stop-history-server.sh. The path to a base log directory + * is must be specified, while the requested UI port is optional. For example: + * + * ./sbin/spark-history-server.sh /tmp/spark-events + * ./sbin/spark-history-server.sh hdfs://1.2.3.4:9000/spark-events + * + * This launches the HistoryServer as a Spark daemon. + */ +object HistoryServer { + private val conf = new SparkConf + + // Interval between each check for event log updates + val UPDATE_INTERVAL_MS = conf.getInt("spark.history.updateInterval", 10) * 1000 + + // How many applications to retain + val RETAINED_APPLICATIONS = conf.getInt("spark.history.retainedApplications", 250) + + // The port to which the web UI is bound + val WEB_UI_PORT = conf.getInt("spark.history.ui.port", 18080) + + // set whether to enable or disable view acls for all applications + val HISTORY_UI_ACLS_ENABLED = conf.getBoolean("spark.history.ui.acls.enable", false) + + val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR + + def main(argStrings: Array[String]) { + initSecurity() + val args = new HistoryServerArguments(argStrings) + val securityManager = new SecurityManager(conf) + val server = new HistoryServer(args.logDir, securityManager, conf) + server.bind() + + // Wait until the end of the world... or if the HistoryServer process is manually stopped + while(true) { Thread.sleep(Int.MaxValue) } + server.stop() + } + + def initSecurity() { + // If we are accessing HDFS and it has security enabled (Kerberos), we have to login + // from a keytab file so that we can access HDFS beyond the kerberos ticket expiration. + // As long as it is using Hadoop rpc (hdfs://), a relogin will automatically + // occur from the keytab. + if (conf.getBoolean("spark.history.kerberos.enabled", false)) { + // if you have enabled kerberos the following 2 params must be set + val principalName = conf.get("spark.history.kerberos.principal") + val keytabFilename = conf.get("spark.history.kerberos.keytab") + SparkHadoopUtil.get.loginUserFromKeytab(principalName, keytabFilename) + } + } + +} + + +private[spark] case class ApplicationHistoryInfo( + id: String, + name: String, + startTime: Long, + endTime: Long, + lastUpdated: Long, + sparkUser: String, + logDirPath: Path, + ui: SparkUI) { + def started = startTime != -1 + def completed = endTime != -1 +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala new file mode 100644 index 0000000000000..943c061743dbd --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.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.deploy.history + +import java.net.URI + +import org.apache.hadoop.fs.Path + +import org.apache.spark.util.Utils + +/** + * Command-line parser for the master. + */ +private[spark] class HistoryServerArguments(args: Array[String]) { + var logDir = "" + + parse(args.toList) + + private def parse(args: List[String]): Unit = { + args match { + case ("--dir" | "-d") :: value :: tail => + logDir = value + parse(tail) + + case ("--help" | "-h") :: tail => + printUsageAndExit(0) + + case Nil => + + case _ => + printUsageAndExit(1) + } + validateLogDir() + } + + private def validateLogDir() { + if (logDir == "") { + System.err.println("Logging directory must be specified.") + printUsageAndExit(1) + } + val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) + val path = new Path(logDir) + if (!fileSystem.exists(path)) { + System.err.println("Logging directory specified does not exist: %s".format(logDir)) + printUsageAndExit(1) + } + if (!fileSystem.getFileStatus(path).isDir) { + System.err.println("Logging directory specified is not a directory: %s".format(logDir)) + printUsageAndExit(1) + } + } + + private def printUsageAndExit(exitCode: Int) { + System.err.println( + "Usage: HistoryServer [options]\n" + + "\n" + + "Options:\n" + + " -d DIR, --dir DIR Location of event log files") + System.exit(exitCode) + } +} 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 46b9f4dc7d3ba..72d0589689e71 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 @@ -20,6 +20,7 @@ package org.apache.spark.deploy.master import java.util.Date import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import akka.actor.ActorRef @@ -36,6 +37,7 @@ private[spark] class ApplicationInfo( @transient var state: ApplicationState.Value = _ @transient var executors: mutable.HashMap[Int, ExecutorInfo] = _ + @transient var removedExecutors: ArrayBuffer[ExecutorInfo] = _ @transient var coresGranted: Int = _ @transient var endTime: Long = _ @transient var appSource: ApplicationSource = _ @@ -51,6 +53,7 @@ private[spark] class ApplicationInfo( endTime = -1L appSource = new ApplicationSource(this) nextExecutorId = 0 + removedExecutors = new ArrayBuffer[ExecutorInfo] } private def newExecutorId(useID: Option[Int] = None): Int = { @@ -74,6 +77,7 @@ private[spark] class ApplicationInfo( def removeExecutor(exec: ExecutorInfo) { if (executors.contains(exec.id)) { + removedExecutors += executors(exec.id) executors -= exec.id coresGranted -= exec.cores } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala index 76db61dd619c6..d417070c51016 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala @@ -34,4 +34,19 @@ private[spark] class ExecutorInfo( } def fullId: String = application.id + "/" + id + + override def equals(other: Any): Boolean = { + other match { + case info: ExecutorInfo => + fullId == info.fullId && + worker.id == info.worker.id && + cores == info.cores && + memory == info.memory + case _ => false + } + } + + override def toString: String = fullId + + override def hashCode: Int = toString.hashCode() } 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 95bd62e88db2b..33ffcbd216954 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 @@ -23,12 +23,14 @@ import java.util.Date import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Await import scala.concurrent.duration._ +import scala.language.postfixOps import scala.util.Random import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension +import org.apache.hadoop.fs.FileSystem import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState} @@ -37,7 +39,7 @@ import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.scheduler.ReplayListenerBus +import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus} import org.apache.spark.ui.SparkUI import org.apache.spark.util.{AkkaUtils, Utils} @@ -45,7 +47,8 @@ private[spark] class Master( host: String, port: Int, webUiPort: Int, - val securityMgr: SecurityManager) extends Actor with Logging { + val securityMgr: SecurityManager) + extends Actor with Logging { import context.dispatcher // to use Akka's scheduler.schedule() @@ -71,6 +74,7 @@ private[spark] class Master( var nextAppNumber = 0 val appIdToUI = new HashMap[String, SparkUI] + val fileSystemsUsed = new HashSet[FileSystem] val drivers = new HashSet[DriverInfo] val completedDrivers = new ArrayBuffer[DriverInfo] @@ -100,6 +104,8 @@ private[spark] class Master( var leaderElectionAgent: ActorRef = _ + private var recoveryCompletionTask: Cancellable = _ + // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. @@ -148,7 +154,12 @@ private[spark] class Master( } override def postStop() { + // prevent the CompleteRecovery message sending to restarted master + if (recoveryCompletionTask != null) { + recoveryCompletionTask.cancel() + } webUi.stop() + fileSystemsUsed.foreach(_.close()) masterMetricsSystem.stop() applicationMetricsSystem.stop() persistenceEngine.close() @@ -166,10 +177,13 @@ private[spark] class Master( logInfo("I have been elected leader! New state: " + state) if (state == RecoveryState.RECOVERING) { beginRecovery(storedApps, storedDrivers, storedWorkers) - context.system.scheduler.scheduleOnce(WORKER_TIMEOUT millis) { completeRecovery() } + recoveryCompletionTask = context.system.scheduler.scheduleOnce(WORKER_TIMEOUT millis, self, + CompleteRecovery) } } + case CompleteRecovery => completeRecovery() + case RevokedLeadership => { logError("Leadership has been revoked -- master shutting down.") System.exit(0) @@ -232,8 +246,7 @@ private[spark] class Master( if (waitingDrivers.contains(d)) { waitingDrivers -= d self ! DriverStateChanged(driverId, DriverState.KILLED, None) - } - else { + } else { // We just notify the worker to kill the driver here. The final bookkeeping occurs // on the return path when the worker submits a state change back to the master // to notify it that the driver was successfully killed. @@ -290,10 +303,11 @@ private[spark] class Master( appInfo.removeExecutor(exec) exec.worker.removeExecutor(exec) + val normalExit = exitStatus.exists(_ == 0) // Only retry certain number of times so we don't go into an infinite loop. - if (appInfo.incrementRetryCount < ApplicationState.MAX_NUM_RETRY) { + if (!normalExit && appInfo.incrementRetryCount < ApplicationState.MAX_NUM_RETRY) { schedule() - } else { + } else if (!normalExit) { logError("Application %s with ID %s failed %d times, removing it".format( appInfo.desc.name, appInfo.id, appInfo.retryCount)) removeApplication(appInfo, ApplicationState.FAILED) @@ -461,7 +475,7 @@ private[spark] class Master( * Schedule the currently available resources among waiting apps. This method will be called * every time a new app joins or resource availability changes. */ - def schedule() { + private def schedule() { if (state != RecoveryState.ALIVE) { return } // First schedule drivers, they take strict precedence over applications @@ -481,7 +495,7 @@ private[spark] class Master( // Try to spread out each app among all the nodes, 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(canUse(app, _)).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) @@ -621,7 +635,7 @@ private[spark] class Master( if (completedApps.size >= RETAINED_APPLICATIONS) { val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1) completedApps.take(toRemove).foreach( a => { - appIdToUI.remove(a.id).foreach { ui => webUi.detachUI(ui) } + appIdToUI.remove(a.id).foreach { ui => webUi.detachSparkUI(ui) } applicationMetricsSystem.removeSource(a.appSource) }) completedApps.trimStart(toRemove) @@ -630,11 +644,7 @@ private[spark] class Master( waitingApps -= app // If application events are logged, use them to rebuild the UI - startPersistedSparkUI(app).map { ui => - app.desc.appUiUrl = ui.basePath - appIdToUI(app.id) = ui - webUi.attachUI(ui) - }.getOrElse { + if (!rebuildSparkUI(app)) { // Avoid broken links if the UI is not reconstructed app.desc.appUiUrl = "" } @@ -654,30 +664,34 @@ private[spark] class Master( } /** - * Start a new SparkUI rendered from persisted storage. If this is unsuccessful for any reason, - * return None. Otherwise return the reconstructed UI. + * Rebuild a new SparkUI from the given application's event logs. + * Return whether this is successful. */ - def startPersistedSparkUI(app: ApplicationInfo): Option[SparkUI] = { + def rebuildSparkUI(app: ApplicationInfo): Boolean = { val appName = app.desc.name - val eventLogInfo = app.desc.eventLogInfo.getOrElse { return None } - val eventLogDir = eventLogInfo.logDir - val eventCompressionCodec = eventLogInfo.compressionCodec - val appConf = new SparkConf - eventCompressionCodec.foreach { codec => - appConf.set("spark.eventLog.compress", "true") - appConf.set("spark.io.compression.codec", codec) - } - val replayerBus = new ReplayListenerBus(appConf) - val ui = new SparkUI( - appConf, - replayerBus, - "%s (finished)".format(appName), - "/history/%s".format(app.id)) - - // Do not call ui.bind() to avoid creating a new server for each application - ui.start() - val success = replayerBus.replay(eventLogDir) - if (success) Some(ui) else None + val eventLogDir = app.desc.eventLogDir.getOrElse { return false } + val fileSystem = Utils.getHadoopFileSystem(eventLogDir) + val eventLogInfo = EventLoggingListener.parseLoggingInfo(eventLogDir, fileSystem) + val eventLogPaths = eventLogInfo.logPaths + val compressionCodec = eventLogInfo.compressionCodec + if (!eventLogPaths.isEmpty) { + try { + val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec) + val ui = new SparkUI( + new SparkConf, replayBus, appName + " (completed)", "/history/" + app.id) + replayBus.replay() + app.desc.appUiUrl = ui.basePath + appIdToUI(app.id) = ui + webUi.attachSparkUI(ui) + return true + } catch { + case e: Exception => + logError("Exception in replaying log for application %s (%s)".format(appName, app.id), e) + } + } else { + logWarning("Application %s (%s) has no valid logs: %s".format(appName, app.id, eventLogDir)) + } + false } /** Generate a new app ID given a app's submission date */ 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 cb092cb5d576b..34fa1429c86de 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 @@ -25,18 +25,19 @@ import scala.xml.Node import akka.pattern.ask import org.json4s.JValue -import org.apache.spark.deploy.JsonProtocol +import org.apache.spark.deploy.{ExecutorState, JsonProtocol} import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.ExecutorInfo -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class ApplicationPage(parent: MasterWebUI) { - val master = parent.masterActorRef - val timeout = parent.timeout +private[spark] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") { + + private val master = parent.masterActorRef + private val timeout = parent.timeout /** Executor details for a particular application */ - def renderJson(request: HttpServletRequest): JValue = { + override def renderJson(request: HttpServletRequest): JValue = { val appId = request.getParameter("appId") val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, timeout) @@ -56,47 +57,59 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { }) val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Logs") - val executors = app.executors.values.toSeq - val executorTable = UIUtils.listingTable(executorHeaders, executorRow, executors) + val allExecutors = (app.executors.values ++ app.removedExecutors).toSet.toSeq + // This includes executors that are either still running or have exited cleanly + val executors = allExecutors.filter { exec => + !ExecutorState.isFinished(exec.state) || exec.state == ExecutorState.EXITED + } + val removedExecutors = allExecutors.diff(executors) + val executorsTable = UIUtils.listingTable(executorHeaders, executorRow, executors) + val removedExecutorsTable = UIUtils.listingTable(executorHeaders, executorRow, removedExecutors) val content = -
-
-
    -
  • ID: {app.id}
  • -
  • Name: {app.desc.name}
  • -
  • User: {app.desc.user}
  • -
  • Cores: - { - if (app.desc.maxCores.isEmpty) { - "Unlimited (%s granted)".format(app.coresGranted) - } else { - "%s (%s granted, %s left)".format( - app.desc.maxCores.get, app.coresGranted, app.coresLeft) - } - } -
  • -
  • - Executor Memory: - {Utils.megabytesToString(app.desc.memoryPerSlave)} -
  • -
  • Submit Date: {app.submitDate}
  • -
  • State: {app.state}
  • -
  • Application Detail UI
  • -
-
+
+
+
    +
  • ID: {app.id}
  • +
  • Name: {app.desc.name}
  • +
  • User: {app.desc.user}
  • +
  • Cores: + { + if (app.desc.maxCores.isEmpty) { + "Unlimited (%s granted)".format(app.coresGranted) + } else { + "%s (%s granted, %s left)".format( + app.desc.maxCores.get, app.coresGranted, app.coresLeft) + } + } +
  • +
  • + Executor Memory: + {Utils.megabytesToString(app.desc.memoryPerSlave)} +
  • +
  • Submit Date: {app.submitDate}
  • +
  • State: {app.state}
  • +
  • Application Detail UI
  • +
+
-
-
-

Executor Summary

- {executorTable} -
-
; +
+
+

Executor Summary

+ {executorsTable} + { + if (removedExecutors.nonEmpty) { +

Removed Executors

++ + removedExecutorsTable + } + } +
+
; UIUtils.basicSparkPage(content, "Application: " + app.desc.name) } - def executorRow(executor: ExecutorInfo): Seq[Node] = { + private def executorRow(executor: ExecutorInfo): Seq[Node] = { {executor.id} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala similarity index 91% rename from core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala rename to core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 8c1d6c7cce450..7ca3b08a28728 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -25,17 +25,17 @@ import scala.xml.Node import akka.pattern.ask import org.json4s.JValue -import org.apache.spark.deploy.{JsonProtocol} +import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} -import org.apache.spark.ui.{WebUI, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class IndexPage(parent: MasterWebUI) { - val master = parent.masterActorRef - val timeout = parent.timeout +private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { + private val master = parent.masterActorRef + private val timeout = parent.timeout - def renderJson(request: HttpServletRequest): JValue = { + override def renderJson(request: HttpServletRequest): JValue = { val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, timeout) JsonProtocol.writeMasterState(state) @@ -139,7 +139,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { UIUtils.basicSparkPage(content, "Spark Master at " + state.uri) } - def workerRow(worker: WorkerInfo): Seq[Node] = { + private def workerRow(worker: WorkerInfo): Seq[Node] = { {worker.id} @@ -154,8 +154,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { } - - def appRow(app: ApplicationInfo): Seq[Node] = { + private def appRow(app: ApplicationInfo): Seq[Node] = { {app.id} @@ -169,14 +168,14 @@ private[spark] class IndexPage(parent: MasterWebUI) { {Utils.megabytesToString(app.desc.memoryPerSlave)} - {WebUI.formatDate(app.submitDate)} + {UIUtils.formatDate(app.submitDate)} {app.desc.user} {app.state.toString} - {WebUI.formatDuration(app.duration)} + {UIUtils.formatDuration(app.duration)} } - def driverRow(driver: DriverInfo): Seq[Node] = { + private def driverRow(driver: DriverInfo): Seq[Node] = { {driver.id} {driver.submitDate} 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 bd75b2dfd0e07..a18b39fc95d64 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 @@ -17,13 +17,9 @@ package org.apache.spark.deploy.master.ui -import javax.servlet.http.HttpServletRequest - -import org.eclipse.jetty.servlet.ServletContextHandler - import org.apache.spark.Logging import org.apache.spark.deploy.master.Master -import org.apache.spark.ui.{ServerInfo, SparkUI} +import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -31,72 +27,33 @@ import org.apache.spark.util.{AkkaUtils, Utils} * Web UI server for the standalone master. */ private[spark] -class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { +class MasterWebUI(val master: Master, requestedPort: Int) + extends WebUI(master.securityMgr, requestedPort, master.conf) with Logging { + val masterActorRef = master.self val timeout = AkkaUtils.askTimeout(master.conf) - private val host = Utils.localHostName() - private val port = requestedPort - private val applicationPage = new ApplicationPage(this) - private val indexPage = new IndexPage(this) - private var serverInfo: Option[ServerInfo] = None - - private val handlers: Seq[ServletContextHandler] = { - master.masterMetricsSystem.getServletHandlers ++ - master.applicationMetricsSystem.getServletHandlers ++ - Seq[ServletContextHandler]( - createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static"), - createServletHandler("/app/json", - (request: HttpServletRequest) => applicationPage.renderJson(request), master.securityMgr), - createServletHandler("/app", - (request: HttpServletRequest) => applicationPage.render(request), master.securityMgr), - createServletHandler("/json", - (request: HttpServletRequest) => indexPage.renderJson(request), master.securityMgr), - createServletHandler("/", - (request: HttpServletRequest) => indexPage.render(request), master.securityMgr) - ) - } + initialize() - def bind() { - try { - serverInfo = Some(startJettyServer(host, port, handlers, master.conf)) - logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Master JettyUtils", e) - System.exit(1) - } + /** Initialize all components of the server. */ + def initialize() { + attachPage(new ApplicationPage(this)) + attachPage(new MasterPage(this)) + attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static")) + master.masterMetricsSystem.getServletHandlers.foreach(attachHandler) + master.applicationMetricsSystem.getServletHandlers.foreach(attachHandler) } - def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) - /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ - def attachUI(ui: SparkUI) { + def attachSparkUI(ui: SparkUI) { assert(serverInfo.isDefined, "Master UI must be bound to a server before attaching SparkUIs") - val rootHandler = serverInfo.get.rootHandler - for (handler <- ui.handlers) { - rootHandler.addHandler(handler) - if (!handler.isStarted) { - handler.start() - } - } + ui.getHandlers.foreach(attachHandler) } /** Detach a reconstructed UI from this Master UI. Only valid after bind(). */ - def detachUI(ui: SparkUI) { + def detachSparkUI(ui: SparkUI) { assert(serverInfo.isDefined, "Master UI must be bound to a server before detaching SparkUIs") - val rootHandler = serverInfo.get.rootHandler - for (handler <- ui.handlers) { - if (handler.isStarted) { - handler.stop() - } - rootHandler.removeHandler(handler) - } - } - - def stop() { - assert(serverInfo.isDefined, "Attempted to stop a Master UI that was not bound to a server!") - serverInfo.get.server.stop() + ui.getHandlers.foreach(detachHandler) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 0c761dfc93a1f..4af5bc3afad6c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -46,21 +46,36 @@ object CommandUtils extends Logging { * the way the JAVA_OPTS are assembled there. */ def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = { - val libraryOpts = getEnv("SPARK_LIBRARY_PATH", command) - .map(p => List("-Djava.library.path=" + p)) - .getOrElse(Nil) - val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")) - .map(Utils.splitCommandString).getOrElse(Nil) - val userOpts = getEnv("SPARK_JAVA_OPTS", command).map(Utils.splitCommandString).getOrElse(Nil) val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M") + val extraOpts = command.extraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq()) + + // Exists for backwards compatibility with older Spark versions + val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString) + .getOrElse(Nil) + if (workerLocalOpts.length > 0) { + logWarning("SPARK_JAVA_OPTS was set on the worker. It is deprecated in Spark 1.0.") + logWarning("Set SPARK_LOCAL_DIRS for node-specific storage locations.") + } + + val libraryOpts = + if (command.libraryPathEntries.size > 0) { + val joined = command.libraryPathEntries.mkString(File.pathSeparator) + Seq(s"-Djava.library.path=$joined") + } else { + Seq() + } + + val permGenOpt = Seq("-XX:MaxPermSize=128m") // Figure out our classpath with the external compute-classpath script val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh" val classPath = Utils.executeAndGetOutput( Seq(sparkHome + "/bin/compute-classpath" + ext), extraEnvironment=command.environment) + val userClassPath = command.classPathEntries ++ Seq(classPath) - Seq("-cp", classPath) ++ libraryOpts ++ workerLocalOpts ++ userOpts ++ memoryOpts + Seq("-cp", userClassPath.filterNot(_.isEmpty).mkString(File.pathSeparator)) ++ + permGenOpt ++ libraryOpts ++ extraOpts ++ workerLocalOpts ++ memoryOpts } /** Spawn a thread that will redirect a given stream to a file */ 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 b4df1a0dd4718..662d37871e7a6 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 @@ -20,7 +20,7 @@ package org.apache.spark.deploy.worker import java.io._ import scala.collection.JavaConversions._ -import scala.collection.mutable.Map +import scala.collection.Map import akka.actor.ActorRef import com.google.common.base.Charsets @@ -74,22 +74,28 @@ private[spark] class DriverRunner( // Make sure user application jar is on the classpath // TODO: If we add ability to submit multiple jars they should also be added here - val env = Map(driverDesc.command.environment.toSeq: _*) - env("SPARK_CLASSPATH") = env.getOrElse("SPARK_CLASSPATH", "") + s":$localJarFilename" - val newCommand = Command(driverDesc.command.mainClass, - driverDesc.command.arguments.map(substituteVariables), env) + val classPath = driverDesc.command.classPathEntries ++ Seq(s"$localJarFilename") + val newCommand = Command( + driverDesc.command.mainClass, + driverDesc.command.arguments.map(substituteVariables), + driverDesc.command.environment, + classPath, + driverDesc.command.libraryPathEntries, + driverDesc.command.extraJavaOptions) val command = CommandUtils.buildCommandSeq(newCommand, driverDesc.mem, sparkHome.getAbsolutePath) - launchDriver(command, env, driverDir, driverDesc.supervise) + launchDriver(command, driverDesc.command.environment, driverDir, driverDesc.supervise) } catch { case e: Exception => finalException = Some(e) } val state = - if (killed) { DriverState.KILLED } - else if (finalException.isDefined) { DriverState.ERROR } - else { + if (killed) { + DriverState.KILLED + } else if (finalException.isDefined) { + DriverState.ERROR + } else { finalExitCode match { case Some(0) => DriverState.FINISHED case _ => DriverState.FAILED 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 be15138f62406..05e242e6df702 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 @@ -31,7 +31,7 @@ object DriverWrapper { case workerUrl :: mainClass :: extraArgs => val conf = new SparkConf() val (actorSystem, _) = AkkaUtils.createActorSystem("Driver", - Utils.localHostName(), 0, false, conf, new SecurityManager(conf)) + Utils.localHostName(), 0, conf, new SecurityManager(conf)) actorSystem.actorOf(Props(classOf[WorkerWatcher], workerUrl), name = "workerWatcher") // Delegate to supplied main class 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 2edd921066876..6433aac1c23e0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -23,9 +23,10 @@ import akka.actor.ActorRef import com.google.common.base.Charsets import com.google.common.io.Files -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.deploy.{ApplicationDescription, Command, ExecutorState} import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged +import org.apache.spark.util.logging.FileAppender /** * Manages the execution of one executor process. @@ -42,12 +43,15 @@ private[spark] class ExecutorRunner( val sparkHome: File, val workDir: File, val workerUrl: String, + val conf: SparkConf, var state: ExecutorState.Value) extends Logging { val fullId = appId + "/" + execId var workerThread: Thread = null var process: Process = null + var stdoutAppender: FileAppender = null + var stderrAppender: FileAppender = null // NOTE: This is now redundant with the automated shut-down enforced by the Executor. It might // make sense to remove this in the future. @@ -58,32 +62,43 @@ private[spark] class ExecutorRunner( override def run() { fetchAndRunExecutor() } } workerThread.start() - // Shutdown hook that kills actors on shutdown. shutdownHook = new Thread() { override def run() { - if (process != null) { - logInfo("Shutdown hook killing child process.") - process.destroy() - process.waitFor() - } + killProcess(Some("Worker shutting down")) } } Runtime.getRuntime.addShutdownHook(shutdownHook) } + /** + * kill executor process, wait for exit and notify worker to update resource status + * + * @param message the exception message which caused the executor's death + */ + private def killProcess(message: Option[String]) { + if (process != null) { + logInfo("Killing process!") + process.destroy() + process.waitFor() + if (stdoutAppender != null) { + stdoutAppender.stop() + } + if (stderrAppender != null) { + stderrAppender.stop() + } + val exitCode = process.waitFor() + worker ! ExecutorStateChanged(appId, execId, state, message, Some(exitCode)) + } + } + /** Stop this executor runner, including killing the process it launched */ def kill() { if (workerThread != null) { + // the workerThread will kill the child process when interrupted workerThread.interrupt() workerThread = null - if (process != null) { - logInfo("Killing process!") - process.destroy() - process.waitFor() - } state = ExecutorState.KILLED - worker ! ExecutorStateChanged(appId, execId, state, None, None) Runtime.getRuntime.removeShutdownHook(shutdownHook) } } @@ -99,7 +114,9 @@ private[spark] class ExecutorRunner( def getCommandSeq = { val command = Command(appDesc.command.mainClass, - appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), appDesc.command.environment) + appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), appDesc.command.environment, + appDesc.command.classPathEntries, appDesc.command.libraryPathEntries, + appDesc.command.extraJavaOptions) CommandUtils.buildCommandSeq(command, memory, sparkHome.getAbsolutePath) } @@ -126,37 +143,33 @@ private[spark] class ExecutorRunner( // parent process for the executor command env.put("SPARK_LAUNCH_WITH_SCALA", "0") process = builder.start() - val header = "Spark Executor Command: %s\n%s\n\n".format( command.mkString("\"", "\" \"", "\""), "=" * 40) // Redirect its stdout and stderr to files val stdout = new File(executorDir, "stdout") - CommandUtils.redirectStream(process.getInputStream, stdout) + stdoutAppender = FileAppender(process.getInputStream, stdout, conf) val stderr = new File(executorDir, "stderr") Files.write(header, stderr, Charsets.UTF_8) - CommandUtils.redirectStream(process.getErrorStream, stderr) + stderrAppender = FileAppender(process.getErrorStream, stderr, conf) - // Wait for it to exit; this is actually a bad thing if it happens, because we expect to run - // long-lived processes only. However, in the future, we might restart the executor a few - // times on the same machine. + // Wait for it to exit; executor may exit with code 0 (when driver instructs it to shutdown) + // or with nonzero exit code val exitCode = process.waitFor() - state = ExecutorState.FAILED + state = ExecutorState.EXITED val message = "Command exited with code " + exitCode worker ! ExecutorStateChanged(appId, execId, state, Some(message), Some(exitCode)) } catch { - case interrupted: InterruptedException => + case interrupted: InterruptedException => { logInfo("Runner thread for executor " + fullId + " interrupted") - + state = ExecutorState.KILLED + killProcess(None) + } case e: Exception => { logError("Error running executor", e) - if (process != null) { - process.destroy() - } state = ExecutorState.FAILED - val message = e.getClass + ": " + e.getMessage - worker ! ExecutorStateChanged(appId, execId, state, Some(message), None) + killProcess(Some(e.toString)) } } } 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 8a71ddda4cb5e..a0ecaf709f8e2 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 @@ -23,6 +23,7 @@ import java.util.Date import scala.collection.mutable.HashMap import scala.concurrent.duration._ +import scala.language.postfixOps import akka.actor._ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} @@ -64,8 +65,12 @@ private[spark] class Worker( val REGISTRATION_TIMEOUT = 20.seconds val REGISTRATION_RETRIES = 3 - // Index into masterUrls that we're currently trying to register with. - var masterIndex = 0 + val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", false) + // How often worker will clean up old app folders + 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 + val APP_DATA_RETENTION_SECS = conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600) + val masterLock: Object = new Object() var master: ActorSelection = null @@ -95,6 +100,8 @@ private[spark] class Worker( val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf, securityMgr) val workerSource = new WorkerSource(this) + var registrationRetryTimer: Option[Cancellable] = None + def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed @@ -122,8 +129,8 @@ private[spark] class Worker( host, port, cores, Utils.megabytesToString(memory))) logInfo("Spark home: " + sparkHome) createWorkDir() - webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) + webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) webUi.bind() registerWithMaster() @@ -156,21 +163,22 @@ private[spark] class Worker( def registerWithMaster() { tryRegisterAllMasters() - var retries = 0 - lazy val retryTimer: Cancellable = + registrationRetryTimer = Some { context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) { - retries += 1 - if (registered) { - retryTimer.cancel() - } else if (retries >= REGISTRATION_RETRIES) { - logError("All masters are unresponsive! Giving up.") - System.exit(1) - } else { - tryRegisterAllMasters() + Utils.tryOrExit { + retries += 1 + if (registered) { + registrationRetryTimer.foreach(_.cancel()) + } else if (retries >= REGISTRATION_RETRIES) { + logError("All masters are unresponsive! Giving up.") + System.exit(1) + } else { + tryRegisterAllMasters() + } } } - retryTimer // start timer + } } override def receive = { @@ -179,12 +187,28 @@ private[spark] class Worker( registered = true changeMaster(masterUrl, masterWebUiUrl) context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis, self, SendHeartbeat) + if (CLEANUP_ENABLED) { + context.system.scheduler.schedule(CLEANUP_INTERVAL_MILLIS millis, + CLEANUP_INTERVAL_MILLIS millis, self, WorkDirCleanup) + } case SendHeartbeat => masterLock.synchronized { if (connected) { master ! Heartbeat(workerId) } } + case WorkDirCleanup => + // Spin up a separate thread (in a future) to do the dir cleanup; don't tie up worker actor + val cleanupFuture = concurrent.future { + logInfo("Cleaning up oldest application directories in " + workDir + " ...") + Utils.findOldFiles(workDir, APP_DATA_RETENTION_SECS) + .foreach(Utils.deleteRecursively) + } + cleanupFuture onFailure { + case e: Throwable => + logError("App dir cleanup failed: " + e.getMessage, e) + } + case MasterChanged(masterUrl, masterWebUiUrl) => logInfo("Master has changed, new master is at " + masterUrl) changeMaster(masterUrl, masterWebUiUrl) @@ -211,7 +235,7 @@ private[spark] class Worker( val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_, self, workerId, host, appDesc.sparkHome.map(userSparkHome => new File(userSparkHome)).getOrElse(sparkHome), - workDir, akkaUrl, ExecutorState.RUNNING) + workDir, akkaUrl, conf, ExecutorState.RUNNING) executors(appId + "/" + execId) = manager manager.start() coresUsed += cores_ @@ -221,7 +245,7 @@ private[spark] class Worker( } } catch { case e: Exception => { - logError("Failed to launch exector %s/%d for %s".format(appId, execId, appDesc.name)) + logError("Failed to launch executor %s/%d for %s".format(appId, execId, appDesc.name)) if (executors.contains(appId + "/" + execId)) { executors(appId + "/" + execId).kill() executors -= appId + "/" + execId @@ -239,14 +263,20 @@ private[spark] class Worker( } val fullId = appId + "/" + execId if (ExecutorState.isFinished(state)) { - val executor = executors(fullId) - logInfo("Executor " + fullId + " finished with state " + state + - message.map(" message " + _).getOrElse("") + - exitStatus.map(" exitStatus " + _).getOrElse("")) - executors -= fullId - finishedExecutors(fullId) = executor - coresUsed -= executor.cores - memoryUsed -= executor.memory + executors.get(fullId) match { + case Some(executor) => + logInfo("Executor " + fullId + " finished with state " + state + + message.map(" message " + _).getOrElse("") + + exitStatus.map(" exitStatus " + _).getOrElse("")) + executors -= fullId + finishedExecutors(fullId) = executor + coresUsed -= executor.cores + memoryUsed -= executor.memory + case None => + logInfo("Unknown Executor " + fullId + " finished with state " + state + + message.map(" message " + _).getOrElse("") + + exitStatus.map(" exitStatus " + _).getOrElse("")) + } } case KillExecutor(masterUrl, appId, execId) => @@ -287,10 +317,14 @@ private[spark] class Worker( state match { case DriverState.ERROR => logWarning(s"Driver $driverId failed with unrecoverable exception: ${exception.get}") + case DriverState.FAILED => + logWarning(s"Driver $driverId exited with failure") case DriverState.FINISHED => logInfo(s"Driver $driverId exited successfully") case DriverState.KILLED => logInfo(s"Driver $driverId was killed by user") + case _ => + logDebug(s"Driver $driverId changed state to $state") } masterLock.synchronized { master ! DriverStateChanged(driverId, state, exception) @@ -323,6 +357,7 @@ private[spark] class Worker( } override def postStop() { + registrationRetryTimer.foreach(_.cancel()) executors.values.foreach(_.kill()) drivers.values.foreach(_.kill()) webUi.stop() @@ -331,7 +366,6 @@ private[spark] class Worker( } private[spark] object Worker { - def main(argStrings: Array[String]) { val args = new WorkerArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, 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 d35d5be73ff97..dc5158102054e 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 @@ -22,7 +22,7 @@ import java.lang.management.ManagementFactory import org.apache.spark.util.{IntParam, MemoryParam, Utils} /** - * Command-line parser for the master. + * Command-line parser for the worker. */ private[spark] class WorkerArguments(args: Array[String]) { var host = Utils.localHostName() @@ -32,8 +32,8 @@ private[spark] class WorkerArguments(args: Array[String]) { var memory = inferDefaultMemory() var masters: Array[String] = null var workDir: String = null - - // Check for settings in environment variables + + // Check for settings in environment variables if (System.getenv("SPARK_WORKER_PORT") != null) { port = System.getenv("SPARK_WORKER_PORT").toInt } @@ -49,7 +49,7 @@ private[spark] class WorkerArguments(args: Array[String]) { if (System.getenv("SPARK_WORKER_DIR") != null) { workDir = System.getenv("SPARK_WORKER_DIR") } - + parse(args.toList) def parse(args: List[String]): Unit = args match { @@ -78,7 +78,7 @@ private[spark] class WorkerArguments(args: Array[String]) { case ("--work-dir" | "-d") :: value :: tail => workDir = value parse(tail) - + case "--webui-port" :: IntParam(value) :: tail => webUiPort = value parse(tail) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala new file mode 100644 index 0000000000000..6a5ffb1b71bfb --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.worker.ui + +import java.io.File +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.spark.ui.{WebUIPage, UIUtils} +import org.apache.spark.util.Utils +import org.apache.spark.Logging +import org.apache.spark.util.logging.{FileAppender, RollingFileAppender} + +private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with Logging { + private val worker = parent.worker + private val workDir = parent.workDir + + def renderLog(request: HttpServletRequest): String = { + val defaultBytes = 100 * 1024 + + val appId = Option(request.getParameter("appId")) + val executorId = Option(request.getParameter("executorId")) + val driverId = Option(request.getParameter("driverId")) + val logType = request.getParameter("logType") + val offset = Option(request.getParameter("offset")).map(_.toLong) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) + + val logDir = (appId, executorId, driverId) match { + case (Some(a), Some(e), None) => + s"${workDir.getPath}/$appId/$executorId/" + case (None, None, Some(d)) => + s"${workDir.getPath}/$driverId/" + case _ => + throw new Exception("Request must specify either application or driver identifiers") + } + + val (logText, startByte, endByte, logLength) = getLog(logDir, logType, offset, byteLength) + val pre = s"==== Bytes $startByte-$endByte of $logLength of $logDir$logType ====\n" + pre + logText + } + + def render(request: HttpServletRequest): Seq[Node] = { + val defaultBytes = 100 * 1024 + val appId = Option(request.getParameter("appId")) + val executorId = Option(request.getParameter("executorId")) + val driverId = Option(request.getParameter("driverId")) + val logType = request.getParameter("logType") + val offset = Option(request.getParameter("offset")).map(_.toLong) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) + + val (logDir, params) = (appId, executorId, driverId) match { + case (Some(a), Some(e), None) => + (s"${workDir.getPath}/$a/$e/", s"appId=$a&executorId=$e") + case (None, None, Some(d)) => + (s"${workDir.getPath}/$d/", s"driverId=$d") + case _ => + throw new Exception("Request must specify either application or driver identifiers") + } + + val (logText, startByte, endByte, logLength) = getLog(logDir, logType, offset, byteLength) + val linkToMaster =

Back to Master

+ val range = Bytes {startByte.toString} - {endByte.toString} of {logLength} + + val backButton = + if (startByte > 0) { + + + + } else { + + } + + val nextButton = + if (endByte < logLength) { + + + + } else { + + } + + val content = + + + {linkToMaster} +
+
{backButton}
+
{range}
+
{nextButton}
+
+
+
+
{logText}
+
+ + + UIUtils.basicSparkPage(content, logType + " log page for " + appId) + } + + /** Get the part of the log files given the offset and desired length of bytes */ + private def getLog( + logDirectory: String, + logType: String, + offsetOption: Option[Long], + byteLength: Int + ): (String, Long, Long, Long) = { + try { + val files = RollingFileAppender.getSortedRolledOverFiles(logDirectory, logType) + logDebug(s"Sorted log files of type $logType in $logDirectory:\n${files.mkString("\n")}") + + val totalLength = files.map { _.length }.sum + val offset = offsetOption.getOrElse(totalLength - byteLength) + val startIndex = { + if (offset < 0) { + 0L + } else if (offset > totalLength) { + totalLength + } else { + offset + } + } + val endIndex = math.min(startIndex + totalLength, totalLength) + logDebug(s"Getting log from $startIndex to $endIndex") + val logText = Utils.offsetBytes(files, startIndex, endIndex) + logDebug(s"Got log of length ${logText.length} bytes") + (logText, startIndex, endIndex, totalLength) + } catch { + case e: Exception => + logError(s"Error getting $logType logs from directory $logDirectory", e) + ("Error getting logs due to exception: " + e.getMessage, 0, 0, 0) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala similarity index 64% rename from core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala rename to core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala index 85200ab0e102d..327b905032800 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala @@ -28,15 +28,15 @@ import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class IndexPage(parent: WorkerWebUI) { +private[spark] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") { val workerActor = parent.worker.self val worker = parent.worker val timeout = parent.timeout - def renderJson(request: HttpServletRequest): JValue = { + override def renderJson(request: HttpServletRequest): JValue = { val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] val workerState = Await.result(stateFuture, timeout) JsonProtocol.writeWorkerState(workerState) @@ -46,74 +46,62 @@ private[spark] class IndexPage(parent: WorkerWebUI) { val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] val workerState = Await.result(stateFuture, timeout) - val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs") + val executorHeaders = Seq("ExecutorID", "Cores", "State", "Memory", "Job Details", "Logs") + val runningExecutors = workerState.executors val runningExecutorTable = - UIUtils.listingTable(executorHeaders, executorRow, workerState.executors) + UIUtils.listingTable(executorHeaders, executorRow, runningExecutors) + val finishedExecutors = workerState.finishedExecutors val finishedExecutorTable = - UIUtils.listingTable(executorHeaders, executorRow, workerState.finishedExecutors) + UIUtils.listingTable(executorHeaders, executorRow, finishedExecutors) val driverHeaders = Seq("DriverID", "Main Class", "State", "Cores", "Memory", "Logs", "Notes") val runningDrivers = workerState.drivers.sortBy(_.driverId).reverse val runningDriverTable = UIUtils.listingTable(driverHeaders, driverRow, runningDrivers) val finishedDrivers = workerState.finishedDrivers.sortBy(_.driverId).reverse - def finishedDriverTable = UIUtils.listingTable(driverHeaders, driverRow, finishedDrivers) + val finishedDriverTable = UIUtils.listingTable(driverHeaders, driverRow, finishedDrivers) // 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 = runningDrivers.length > 0 || finishedDrivers.length > 0 val content = -
-
-
    -
  • ID: {workerState.workerId}
  • -
  • - Master URL: {workerState.masterUrl} -
  • -
  • Cores: {workerState.cores} ({workerState.coresUsed} Used)
  • -
  • Memory: {Utils.megabytesToString(workerState.memory)} - ({Utils.megabytesToString(workerState.memoryUsed)} Used)
  • -
-

Back to Master

-
+
+
+
    +
  • ID: {workerState.workerId}
  • +
  • + Master URL: {workerState.masterUrl} +
  • +
  • Cores: {workerState.cores} ({workerState.coresUsed} Used)
  • +
  • Memory: {Utils.megabytesToString(workerState.memory)} + ({Utils.megabytesToString(workerState.memoryUsed)} Used)
  • +
+

Back to Master

- -
-
-

Running Executors {workerState.executors.size}

- {runningExecutorTable} -
-
- // scalastyle:off -
- {if (hasDrivers) -
-
-

Running Drivers {workerState.drivers.size}

- {runningDriverTable} -
-
+
+
+
+

Running Executors ({runningExecutors.size})

+ {runningExecutorTable} + { + if (runningDrivers.nonEmpty) { +

Running Drivers ({runningDrivers.size})

++ + runningDriverTable + } } -
- -
-
-

Finished Executors

- {finishedExecutorTable} -
-
- -
- {if (hasDrivers) -
-
-

Finished Drivers

- {finishedDriverTable} -
-
+ { + if (finishedExecutors.nonEmpty) { +

Finished Executors ({finishedExecutors.size})

++ + finishedExecutorTable + } + } + { + if (finishedDrivers.nonEmpty) { +

Finished Drivers ({finishedDrivers.size})

++ + finishedDriverTable + } } -
; - // scalastyle:on +
+
; UIUtils.basicSparkPage(content, "Spark Worker at %s:%s".format( workerState.host, workerState.port)) } @@ -122,6 +110,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { {executor.execId} {executor.cores} + {executor.state} {Utils.megabytesToString(executor.memory)} @@ -137,7 +126,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { .format(executor.appId, executor.execId)}>stdout stderr - + } 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 de76a5d5eb7bc..0ad2edba2227f 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 @@ -20,180 +20,44 @@ package org.apache.spark.deploy.worker.ui import java.io.File import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.servlet.ServletContextHandler - -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.worker.Worker -import org.apache.spark.ui.{JettyUtils, ServerInfo, SparkUI, UIUtils} +import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.AkkaUtils /** * Web UI server for the standalone worker. */ private[spark] -class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None) - extends Logging { +class WorkerWebUI( + val worker: Worker, + val workDir: File, + port: Option[Int] = None) + extends WebUI(worker.securityMgr, WorkerWebUI.getUIPort(port, worker.conf), worker.conf) + with Logging { val timeout = AkkaUtils.askTimeout(worker.conf) - private val host = Utils.localHostName() - private val port = requestedPort.getOrElse( - worker.conf.get("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) - private val indexPage = new IndexPage(this) - private var serverInfo: Option[ServerInfo] = None - - private val handlers: Seq[ServletContextHandler] = { - worker.metricsSystem.getServletHandlers ++ - Seq[ServletContextHandler]( - createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE, "/static"), - createServletHandler("/log", - (request: HttpServletRequest) => log(request), worker.securityMgr), - createServletHandler("/logPage", - (request: HttpServletRequest) => logPage(request), worker.securityMgr), - createServletHandler("/json", - (request: HttpServletRequest) => indexPage.renderJson(request), worker.securityMgr), - createServletHandler("/", - (request: HttpServletRequest) => indexPage.render(request), worker.securityMgr) - ) - } - - def bind() { - try { - serverInfo = Some(JettyUtils.startJettyServer(host, port, handlers, worker.conf)) - logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Worker JettyUtils", e) - System.exit(1) - } - } - - def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) - - private def log(request: HttpServletRequest): String = { - val defaultBytes = 100 * 1024 - - val appId = Option(request.getParameter("appId")) - val executorId = Option(request.getParameter("executorId")) - val driverId = Option(request.getParameter("driverId")) - val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toLong) - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) - - val path = (appId, executorId, driverId) match { - case (Some(a), Some(e), None) => - s"${workDir.getPath}/$appId/$executorId/$logType" - case (None, None, Some(d)) => - s"${workDir.getPath}/$driverId/$logType" - case _ => - throw new Exception("Request must specify either application or driver identifiers") - } - - val (startByte, endByte) = getByteRange(path, offset, byteLength) - val file = new File(path) - val logLength = file.length - - val pre = s"==== Bytes $startByte-$endByte of $logLength of $path ====\n" - pre + Utils.offsetBytes(path, startByte, endByte) - } - - private def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { - val defaultBytes = 100 * 1024 - val appId = Option(request.getParameter("appId")) - val executorId = Option(request.getParameter("executorId")) - val driverId = Option(request.getParameter("driverId")) - val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toLong) - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) - - val (path, params) = (appId, executorId, driverId) match { - case (Some(a), Some(e), None) => - (s"${workDir.getPath}/$a/$e/$logType", s"appId=$a&executorId=$e") - case (None, None, Some(d)) => - (s"${workDir.getPath}/$d/$logType", s"driverId=$d") - case _ => - throw new Exception("Request must specify either application or driver identifiers") - } - - val (startByte, endByte) = getByteRange(path, offset, byteLength) - val file = new File(path) - val logLength = file.length - val logText = {Utils.offsetBytes(path, startByte, endByte)} - val linkToMaster =

Back to Master

- val range = Bytes {startByte.toString} - {endByte.toString} of {logLength} - - val backButton = - if (startByte > 0) { - - - - } - else { - - } - - val nextButton = - if (endByte < logLength) { - - - - } - else { - - } - - val content = - - - {linkToMaster} -
-
{backButton}
-
{range}
-
{nextButton}
-
-
-
-
{logText}
-
- - - UIUtils.basicSparkPage(content, logType + " log page for " + appId) - } - - /** Determine the byte range for a log or log page. */ - private def getByteRange(path: String, offset: Option[Long], byteLength: Int): (Long, Long) = { - val defaultBytes = 100 * 1024 - val maxBytes = 1024 * 1024 - val file = new File(path) - val logLength = file.length() - val getOffset = offset.getOrElse(logLength - defaultBytes) - val startByte = - if (getOffset < 0) 0L - else if (getOffset > logLength) logLength - else getOffset - val logPageLength = math.min(byteLength, maxBytes) - val endByte = math.min(startByte + logPageLength, logLength) - (startByte, endByte) - } - - def stop() { - assert(serverInfo.isDefined, "Attempted to stop a Worker UI that was not bound to a server!") - serverInfo.get.server.stop() + initialize() + + /** Initialize all components of the server. */ + def initialize() { + val logPage = new LogPage(this) + attachPage(logPage) + attachPage(new WorkerPage(this)) + attachHandler(createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE, "/static")) + attachHandler(createServletHandler("/log", + (request: HttpServletRequest) => logPage.renderLog(request), worker.securityMgr)) + worker.metricsSystem.getServletHandlers.foreach(attachHandler) } } private[spark] object WorkerWebUI { + val DEFAULT_PORT = 8081 val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR - val DEFAULT_PORT="8081" + + def getUIPort(requestedPort: Option[Int], conf: SparkConf): Int = { + requestedPort.getOrElse(conf.getInt("worker.ui.port", WorkerWebUI.DEFAULT_PORT)) + } } 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 16887d8892b31..2279d77c91c89 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -22,10 +22,12 @@ import java.nio.ByteBuffer import akka.actor._ import akka.remote._ -import org.apache.spark.{SecurityManager, SparkConf, Logging} +import org.apache.spark.{SparkEnv, Logging, SecurityManager, SparkConf} import org.apache.spark.TaskState.TaskState +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.worker.WorkerWatcher import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ +import org.apache.spark.scheduler.TaskDescription import org.apache.spark.util.{AkkaUtils, Utils} private[spark] class CoarseGrainedExecutorBackend( @@ -53,28 +55,30 @@ private[spark] class CoarseGrainedExecutorBackend( case RegisteredExecutor(sparkProperties) => logInfo("Successfully registered with driver") // Make this host instead of hostPort ? - executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties, + executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties, false) case RegisterExecutorFailed(message) => logError("Slave registration failed: " + message) System.exit(1) - case LaunchTask(taskDesc) => - logInfo("Got assigned task " + taskDesc.taskId) + case LaunchTask(data) => if (executor == null) { logError("Received LaunchTask command but executor was null") System.exit(1) } else { + val ser = SparkEnv.get.closureSerializer.newInstance() + val taskDesc = ser.deserialize[TaskDescription](data.value) + logInfo("Got assigned task " + taskDesc.taskId) executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask) } - case KillTask(taskId, _) => + case KillTask(taskId, _, interruptThread) => if (executor == null) { logError("Received KillTask command but executor was null") System.exit(1) } else { - executor.killTask(taskId) + executor.killTask(taskId, interruptThread) } case x: DisassociatedEvent => @@ -94,25 +98,30 @@ private[spark] class CoarseGrainedExecutorBackend( private[spark] object CoarseGrainedExecutorBackend { def run(driverUrl: String, executorId: String, hostname: String, cores: Int, - workerUrl: Option[String]) { - // Debug code - Utils.checkHost(hostname) - - val conf = new SparkConf - // Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor - // before getting started with all our system properties, etc - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0, - indestructible = true, conf = conf, new SecurityManager(conf)) - // set it - val sparkHostPort = hostname + ":" + boundPort - actorSystem.actorOf( - Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, - sparkHostPort, cores), - name = "Executor") - workerUrl.foreach{ url => - actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") + workerUrl: Option[String]) { + + SparkHadoopUtil.get.runAsSparkUser { () => + // Debug code + Utils.checkHost(hostname) + + val conf = new SparkConf + // Create a new ActorSystem to run the backend, because we can't create a + // SparkEnv / Executor before getting started with all our system properties, etc + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0, + conf, new SecurityManager(conf)) + // set it + val sparkHostPort = hostname + ":" + boundPort + actorSystem.actorOf( + Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, + sparkHostPort, cores), + name = "Executor") + workerUrl.foreach { + url => + actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") + } + actorSystem.awaitTermination() + } - actorSystem.awaitTermination() } def main(args: Array[String]) { 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 aecb069e4202b..baee7a216a7c3 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -64,37 +64,17 @@ private[spark] class Executor( // to what Yarn on this system said was available. This will be used later when SparkEnv // created. if (java.lang.Boolean.valueOf( - System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))) - { + System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))) { conf.set("spark.local.dir", getYarnLocalDirs()) + } else if (sys.env.contains("SPARK_LOCAL_DIRS")) { + conf.set("spark.local.dir", sys.env("SPARK_LOCAL_DIRS")) } if (!isLocal) { // Setup an uncaught exception handler for non-local mode. // Make any thread terminations due to uncaught exceptions kill the entire // executor process to avoid surprising stalls. - Thread.setDefaultUncaughtExceptionHandler( - new Thread.UncaughtExceptionHandler { - override def uncaughtException(thread: Thread, exception: Throwable) { - try { - logError("Uncaught exception in thread " + thread, exception) - - // We may have been called from a shutdown hook. If so, we must not call System.exit(). - // (If we do, we will deadlock.) - if (!Utils.inShutdown()) { - if (exception.isInstanceOf[OutOfMemoryError]) { - System.exit(ExecutorExitCode.OOM) - } else { - System.exit(ExecutorExitCode.UNCAUGHT_EXCEPTION) - } - } - } catch { - case oom: OutOfMemoryError => Runtime.getRuntime.halt(ExecutorExitCode.OOM) - case t: Throwable => Runtime.getRuntime.halt(ExecutorExitCode.UNCAUGHT_EXCEPTION_TWICE) - } - } - } - ) + Thread.setDefaultUncaughtExceptionHandler(ExecutorUncaughtExceptionHandler) } val executorSource = new ExecutorSource(this, executorId) @@ -127,18 +107,16 @@ private[spark] class Executor( // Maintains the list of running tasks. private val runningTasks = new ConcurrentHashMap[Long, TaskRunner] - val sparkUser = Option(System.getenv("SPARK_USER")).getOrElse(SparkContext.SPARK_UNKNOWN_USER) - def launchTask(context: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer) { val tr = new TaskRunner(context, taskId, serializedTask) runningTasks.put(taskId, tr) threadPool.execute(tr) } - def killTask(taskId: Long) { + def killTask(taskId: Long, interruptThread: Boolean) { val tr = runningTasks.get(taskId) if (tr != null) { - tr.kill() + tr.kill(interruptThread) } } @@ -160,20 +138,18 @@ private[spark] class Executor( class TaskRunner(execBackend: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer) extends Runnable { - object TaskKilledException extends Exception - @volatile private var killed = false @volatile private var task: Task[Any] = _ - def kill() { + def kill(interruptThread: Boolean) { logInfo("Executor is trying to kill task " + taskId) killed = true if (task != null) { - task.kill() + task.kill(interruptThread) } } - override def run(): Unit = SparkHadoopUtil.get.runAsUser(sparkUser) { () => + override def run() { val startTime = System.currentTimeMillis() SparkEnv.set(env) Thread.currentThread.setContextClassLoader(replClassLoader) @@ -199,7 +175,7 @@ private[spark] class Executor( // causes a NonLocalReturnControl exception to be thrown. The NonLocalReturnControl // exception will be caught by the catch block, leading to an incorrect ExceptionFailure // for the task. - throw TaskKilledException + throw new TaskKilledException } attemptedTask = Some(task) @@ -213,7 +189,7 @@ private[spark] class Executor( // If the task has been killed, let's fail it. if (task.killed) { - throw TaskKilledException + throw new TaskKilledException } val resultSer = SparkEnv.get.serializer.newInstance() @@ -256,12 +232,17 @@ private[spark] class Executor( execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) } - case TaskKilledException => { + case _: TaskKilledException | _: InterruptedException if task.killed => { logInfo("Executor killed task " + taskId) execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) } case t: Throwable => { + // Attempt to exit cleanly by informing the driver of our failure. + // If anything goes wrong (or this was a fatal exception), we will delegate to + // the default uncaught exception handler, which will terminate the Executor. + logError("Exception in task ID " + taskId, t) + val serviceTime = System.currentTimeMillis() - taskStart val metrics = attemptedTask.flatMap(t => t.metrics) for (m <- metrics) { @@ -271,10 +252,11 @@ private[spark] class Executor( val reason = ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace, metrics) execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) - // TODO: Should we exit the whole executor here? On the one hand, the failed task may - // have left some weird state around depending on when the exception was thrown, but on - // the other hand, maybe we could detect that when future tasks fail and exit then. - logError("Exception in task ID " + taskId, t) + // Don't forcibly exit unless the exception was inherently fatal, to avoid + // stopping other tasks unnecessarily. + if (Utils.isFatalError(t)) { + ExecutorUncaughtExceptionHandler.uncaughtException(t) + } } } finally { // TODO: Unregister shuffle memory only for ResultTask @@ -291,15 +273,19 @@ private[spark] class Executor( * Create a ClassLoader for use in tasks, adding any JARs specified by the user or any classes * created by the interpreter to the search path */ - private def createClassLoader(): ExecutorURLClassLoader = { - val loader = Thread.currentThread().getContextClassLoader + private def createClassLoader(): MutableURLClassLoader = { + val currentLoader = Utils.getContextOrSparkClassLoader // For each of the jars in the jarSet, add them to the class loader. // We assume each of the files has already been fetched. val urls = currentJars.keySet.map { uri => new File(uri.split("/").last).toURI.toURL }.toArray - new ExecutorURLClassLoader(urls, loader) + val userClassPathFirst = conf.getBoolean("spark.files.userClassPathFirst", false) + userClassPathFirst match { + case true => new ChildExecutorURLClassLoader(urls, currentLoader) + case false => new ExecutorURLClassLoader(urls, currentLoader) + } } /** @@ -310,11 +296,14 @@ private[spark] class Executor( val classUri = conf.get("spark.repl.class.uri", null) if (classUri != null) { logInfo("Using REPL class URI: " + classUri) + val userClassPathFirst: java.lang.Boolean = + conf.getBoolean("spark.files.userClassPathFirst", false) try { val klass = Class.forName("org.apache.spark.repl.ExecutorClassLoader") .asInstanceOf[Class[_ <: ClassLoader]] - val constructor = klass.getConstructor(classOf[String], classOf[ClassLoader]) - constructor.newInstance(classUri, parent) + val constructor = klass.getConstructor(classOf[String], classOf[ClassLoader], + classOf[Boolean]) + constructor.newInstance(classUri, parent, userClassPathFirst) } catch { case _: ClassNotFoundException => logError("Could not find org.apache.spark.repl.ExecutorClassLoader on classpath!") diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala index 3d34960653f5d..e07cb31cbe4ba 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala @@ -27,3 +27,4 @@ import org.apache.spark.TaskState.TaskState private[spark] trait ExecutorBackend { def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) } + 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 ceff3a067d72a..38be2c58b333f 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala @@ -34,7 +34,7 @@ object ExecutorExitCode { logging the exception. */ val UNCAUGHT_EXCEPTION_TWICE = 51 - /** The default uncaught exception handler was reached, and the uncaught exception was an + /** The default uncaught exception handler was reached, and the uncaught exception was an OutOfMemoryError. */ val OOM = 52 @@ -43,10 +43,10 @@ object ExecutorExitCode { /** TachyonStore failed to initialize after many attempts. */ val TACHYON_STORE_FAILED_TO_INITIALIZE = 54 - + /** TachyonStore failed to create a local temporary directory after many attempts. */ val TACHYON_STORE_FAILED_TO_CREATE_DIR = 55 - + def explainExitCode(exitCode: Int): String = { exitCode match { case UNCAUGHT_EXCEPTION => "Uncaught exception" @@ -57,7 +57,7 @@ object ExecutorExitCode { 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." - case _ => + case _ => "Unknown executor exit code (" + exitCode + ")" + ( if (exitCode > 128) { " (died from signal " + (exitCode - 128) + "?)" diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index 127f5e90f3e1a..0ed52cfe9df61 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.fs.FileSystem import org.apache.spark.metrics.source.Source -class ExecutorSource(val executor: Executor, executorId: String) extends Source { +private[spark] class ExecutorSource(val executor: Executor, executorId: String) extends Source { private def fileStats(scheme: String) : Option[FileSystem.Statistics] = FileSystem.getAllStatistics().filter(s => s.getScheme.equals(scheme)).headOption diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala index f9bfe8ed2f5ba..218ed7b5d2d39 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala @@ -19,13 +19,56 @@ package org.apache.spark.executor import java.net.{URLClassLoader, URL} +import org.apache.spark.util.ParentClassLoader + /** * The addURL method in URLClassLoader is protected. We subclass it to make this accessible. + * We also make changes so user classes can come before the default classes. */ + +private[spark] trait MutableURLClassLoader extends ClassLoader { + def addURL(url: URL) + def getURLs: Array[URL] +} + +private[spark] class ChildExecutorURLClassLoader(urls: Array[URL], parent: ClassLoader) + extends MutableURLClassLoader { + + private object userClassLoader extends URLClassLoader(urls, null){ + override def addURL(url: URL) { + super.addURL(url) + } + override def findClass(name: String): Class[_] = { + super.findClass(name) + } + } + + private val parentClassLoader = new ParentClassLoader(parent) + + override def findClass(name: String): Class[_] = { + try { + userClassLoader.findClass(name) + } catch { + case e: ClassNotFoundException => { + parentClassLoader.loadClass(name) + } + } + } + + def addURL(url: URL) { + userClassLoader.addURL(url) + } + + def getURLs() = { + userClassLoader.getURLs() + } +} + private[spark] class ExecutorURLClassLoader(urls: Array[URL], parent: ClassLoader) - extends URLClassLoader(urls, parent) { + extends URLClassLoader(urls, parent) with MutableURLClassLoader { override def addURL(url: URL) { super.addURL(url) } } + diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala new file mode 100644 index 0000000000000..b0e984c03964c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.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.executor + +import org.apache.spark.Logging +import org.apache.spark.util.Utils + +/** + * The default uncaught exception handler for Executors terminates the whole process, to avoid + * getting into a bad state indefinitely. Since Executors are relatively lightweight, it's better + * to fail fast when things go wrong. + */ +private[spark] object ExecutorUncaughtExceptionHandler + extends Thread.UncaughtExceptionHandler with Logging { + + override def uncaughtException(thread: Thread, exception: Throwable) { + try { + logError("Uncaught exception in thread " + thread, exception) + + // We may have been called from a shutdown hook. If so, we must not call System.exit(). + // (If we do, we will deadlock.) + if (!Utils.inShutdown()) { + if (exception.isInstanceOf[OutOfMemoryError]) { + System.exit(ExecutorExitCode.OOM) + } else { + System.exit(ExecutorExitCode.UNCAUGHT_EXCEPTION) + } + } + } catch { + case oom: OutOfMemoryError => Runtime.getRuntime.halt(ExecutorExitCode.OOM) + case t: Throwable => Runtime.getRuntime.halt(ExecutorExitCode.UNCAUGHT_EXCEPTION_TWICE) + } + } + + def uncaughtException(exception: Throwable) { + uncaughtException(Thread.currentThread(), exception) + } +} diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 6fc702fdb1512..74100498bb2bd 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -19,14 +19,14 @@ package org.apache.spark.executor import java.nio.ByteBuffer -import com.google.protobuf.ByteString +import org.apache.mesos.protobuf.ByteString import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver, MesosNativeLibrary} import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _} -import org.apache.spark.Logging -import org.apache.spark.TaskState +import org.apache.spark.{Logging, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.util.Utils +import org.apache.spark.deploy.SparkHadoopUtil private[spark] class MesosExecutorBackend extends MesosExecutor @@ -76,7 +76,8 @@ private[spark] class MesosExecutorBackend if (executor == null) { logError("Received KillTask but executor was null") } else { - executor.killTask(t.getValue.toLong) + // TODO: Determine the 'interruptOnCancel' property set for the given job. + executor.killTask(t.getValue.toLong, interruptThread = false) } } @@ -94,9 +95,11 @@ private[spark] class MesosExecutorBackend */ private[spark] object MesosExecutorBackend { def main(args: Array[String]) { - MesosNativeLibrary.load() - // Create a new Executor and start it running - val runner = new MesosExecutorBackend() - new MesosExecutorDriver(runner).run() + SparkHadoopUtil.get.runAsSparkUser { () => + MesosNativeLibrary.load() + // Create a new Executor and start it running + val runner = new MesosExecutorBackend() + new MesosExecutorDriver(runner).run() + } } } 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 88625e79a5c68..350fd74173f65 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,8 +17,14 @@ package org.apache.spark.executor +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.storage.{BlockId, BlockStatus} +/** + * :: DeveloperApi :: + * Metrics tracked during the execution of a task. + */ +@DeveloperApi class TaskMetrics extends Serializable { /** * Host's name the task runs on @@ -77,11 +83,16 @@ class TaskMetrics extends Serializable { var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None } -object TaskMetrics { - private[spark] def empty(): TaskMetrics = new TaskMetrics +private[spark] object TaskMetrics { + def empty: TaskMetrics = new TaskMetrics } +/** + * :: DeveloperApi :: + * Metrics pertaining to shuffle data read in a given task. + */ +@DeveloperApi class ShuffleReadMetrics extends Serializable { /** * Absolute time when this task finished reading shuffle data @@ -116,6 +127,11 @@ class ShuffleReadMetrics extends Serializable { var remoteBytesRead: Long = _ } +/** + * :: DeveloperApi :: + * Metrics pertaining to shuffle data written in a given task. + */ +@DeveloperApi class ShuffleWriteMetrics extends Serializable { /** * Number of bytes written for the shuffle by this task diff --git a/core/src/main/scala/org/apache/spark/executor/package-info.java b/core/src/main/scala/org/apache/spark/executor/package-info.java new file mode 100644 index 0000000000000..dd3b6815fb45f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/package-info.java @@ -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. + */ + +/** + * Package for executor components used with various cluster managers. + */ +package org.apache.spark.executor; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/executor/package.scala b/core/src/main/scala/org/apache/spark/executor/package.scala new file mode 100644 index 0000000000000..ef471d761d4b3 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/package.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 + +/** + * Executor components used with various cluster managers. + * See [[org.apache.spark.executor.Executor]]. + */ +package object executor diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala index 4887fb6b84eb2..4cb450577796a 100644 --- a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala @@ -17,6 +17,8 @@ package org.apache.spark.input +import scala.collection.JavaConversions._ + import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.InputSplit import org.apache.hadoop.mapreduce.JobContext @@ -44,4 +46,17 @@ private[spark] class WholeTextFileInputFormat extends CombineFileInputFormat[Str context, classOf[WholeTextFileRecordReader]) } + + /** + * Allow minPartitions set by end-user in order to keep compatibility with old Hadoop API. + */ + def setMaxSplitSize(context: JobContext, minPartitions: Int) { + val files = listStatus(context) + val totalLen = files.map { file => + if (file.isDir) 0L else file.getLen + }.sum + val maxSplitSize = Math.ceil(totalLen * 1.0 / + (if (minPartitions == 0) 1 else minPartitions)).toLong + super.setMaxSplitSize(maxSplitSize) + } } 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 059e58824c39b..4b0fe1ab82999 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -23,11 +23,19 @@ import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.Utils /** + * :: DeveloperApi :: * CompressionCodec allows the customization of choosing different compression implementations * to be used in block storage. + * + * Note: The wire protocol for a codec is not guaranteed compatible across versions of Spark. + * This is intended for use as an internal compression utility within a single + * Spark application. */ +@DeveloperApi trait CompressionCodec { def compressedOutputStream(s: OutputStream): OutputStream @@ -42,7 +50,7 @@ private[spark] object CompressionCodec { } def createCodec(conf: SparkConf, codecName: String): CompressionCodec = { - val ctor = Class.forName(codecName, true, Thread.currentThread.getContextClassLoader) + val ctor = Class.forName(codecName, true, Utils.getContextOrSparkClassLoader) .getConstructor(classOf[SparkConf]) ctor.newInstance(conf).asInstanceOf[CompressionCodec] } @@ -52,8 +60,14 @@ private[spark] object CompressionCodec { /** + * :: DeveloperApi :: * LZF implementation of [[org.apache.spark.io.CompressionCodec]]. + * + * 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 + * application. */ +@DeveloperApi class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { @@ -65,9 +79,15 @@ 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. + * + * 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 + * application. */ +@DeveloperApi class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { diff --git a/core/src/main/scala/org/apache/spark/io/package-info.java b/core/src/main/scala/org/apache/spark/io/package-info.java new file mode 100644 index 0000000000000..bea1bfdb63751 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/io/package-info.java @@ -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. + */ + +/** + * IO codecs used for compression. + */ +package org.apache.spark.io; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/ui/Page.scala b/core/src/main/scala/org/apache/spark/io/package.scala similarity index 85% rename from core/src/main/scala/org/apache/spark/ui/Page.scala rename to core/src/main/scala/org/apache/spark/io/package.scala index b2a069a37552d..f987e66a5dc44 100644 --- a/core/src/main/scala/org/apache/spark/ui/Page.scala +++ b/core/src/main/scala/org/apache/spark/io/package.scala @@ -15,8 +15,9 @@ * limitations under the License. */ -package org.apache.spark.ui +package org.apache.spark -private[spark] object Page extends Enumeration { - val Stages, Storage, Environment, Executors = Value -} +/** + * IO codecs used for compression. See [[org.apache.spark.io.CompressionCodec]]. + */ +package object io diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala index 3e3e18c3537d0..1b7a5d1f1980a 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala @@ -24,6 +24,7 @@ import scala.collection.mutable import scala.util.matching.Regex import org.apache.spark.Logging +import org.apache.spark.util.Utils private[spark] class MetricsConfig(val configFile: Option[String]) extends Logging { @@ -50,7 +51,7 @@ private[spark] class MetricsConfig(val configFile: Option[String]) extends Loggi try { is = configFile match { case Some(f) => new FileInputStream(f) - case None => getClass.getClassLoader.getResourceAsStream(METRICS_CONF) + case None => Utils.getSparkClassLoader.getResourceAsStream(METRICS_CONF) } if (is != null) { 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 c5bda2078fc14..651511da1b7fe 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -129,17 +129,19 @@ private[spark] class MetricsSystem private (val instance: String, sinkConfigs.foreach { kv => val classPath = kv._2.getProperty("class") - try { - val sink = Class.forName(classPath) - .getConstructor(classOf[Properties], classOf[MetricRegistry], classOf[SecurityManager]) - .newInstance(kv._2, registry, securityMgr) - if (kv._1 == "servlet") { - metricsServlet = Some(sink.asInstanceOf[MetricsServlet]) - } else { - sinks += sink.asInstanceOf[Sink] + if (null != classPath) { + try { + val sink = Class.forName(classPath) + .getConstructor(classOf[Properties], classOf[MetricRegistry], classOf[SecurityManager]) + .newInstance(kv._2, registry, securityMgr) + if (kv._1 == "servlet") { + metricsServlet = Some(sink.asInstanceOf[MetricsServlet]) + } else { + sinks += sink.asInstanceOf[Sink] + } + } catch { + case e: Exception => logError("Sink class " + classPath + " cannot be instantialized", e) } - } catch { - case e: Exception => logError("Sink class " + classPath + " cannot be instantialized", e) } } } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala index 64eac73605388..05852f1f98993 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala @@ -25,7 +25,7 @@ import com.codahale.metrics.{ConsoleReporter, MetricRegistry} import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem -class ConsoleSink(val property: Properties, val registry: MetricRegistry, +private[spark] class ConsoleSink(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val CONSOLE_DEFAULT_PERIOD = 10 val CONSOLE_DEFAULT_UNIT = "SECONDS" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala index 544848d4150b6..542dce65366b2 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala @@ -26,7 +26,7 @@ import com.codahale.metrics.{CsvReporter, MetricRegistry} import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem -class CsvSink(val property: Properties, val registry: MetricRegistry, +private[spark] class CsvSink(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val CSV_KEY_PERIOD = "period" val CSV_KEY_UNIT = "unit" @@ -45,7 +45,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry, case Some(s) => TimeUnit.valueOf(s.toUpperCase()) case None => TimeUnit.valueOf(CSV_DEFAULT_UNIT) } - + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) val pollDir = Option(property.getProperty(CSV_KEY_DIR)) match { diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala index 7f0a2fd16fa99..aeb4ad44a0647 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala @@ -27,7 +27,7 @@ import com.codahale.metrics.graphite.{Graphite, GraphiteReporter} import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem -class GraphiteSink(val property: Properties, val registry: MetricRegistry, +private[spark] class GraphiteSink(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val GRAPHITE_DEFAULT_PERIOD = 10 val GRAPHITE_DEFAULT_UNIT = "SECONDS" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala index 3b5edd5c376f0..ed27234b4e760 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala @@ -22,7 +22,7 @@ import java.util.Properties import com.codahale.metrics.{JmxReporter, MetricRegistry} import org.apache.spark.SecurityManager -class JmxSink(val property: Properties, val registry: MetricRegistry, +private[spark] class JmxSink(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val reporter: JmxReporter = JmxReporter.forRegistry(registry).build() 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 854b52c510e3d..571539ba5e467 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,7 +30,7 @@ import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.SecurityManager import org.apache.spark.ui.JettyUtils._ -class MetricsServlet(val property: Properties, val registry: MetricRegistry, +private[spark] class MetricsServlet(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val SERVLET_KEY_PATH = "path" val SERVLET_KEY_SAMPLE = "sample" 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 3a739aa563eae..6f2b5a06027ea 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 @@ -17,7 +17,7 @@ package org.apache.spark.metrics.sink -trait Sink { +private[spark] trait Sink { def start: Unit def stop: Unit } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/package.scala b/core/src/main/scala/org/apache/spark/metrics/sink/package.scala new file mode 100644 index 0000000000000..90e3aa70b99ef --- /dev/null +++ b/core/src/main/scala/org/apache/spark/metrics/sink/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.metrics + +/** + * Sinks used in Spark's metrics system. + */ +package object sink diff --git a/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala b/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala index 75cb2b8973aa1..f865f9648a91e 100644 --- a/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala @@ -20,7 +20,7 @@ package org.apache.spark.metrics.source import com.codahale.metrics.MetricRegistry import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} -class JvmSource extends Source { +private[spark] class JvmSource extends Source { val sourceName = "jvm" val metricRegistry = new MetricRegistry() diff --git a/core/src/main/scala/org/apache/spark/metrics/source/Source.scala b/core/src/main/scala/org/apache/spark/metrics/source/Source.scala index 3fee55cc6dcd5..1dda2cd83b2a9 100644 --- a/core/src/main/scala/org/apache/spark/metrics/source/Source.scala +++ b/core/src/main/scala/org/apache/spark/metrics/source/Source.scala @@ -19,7 +19,7 @@ package org.apache.spark.metrics.source import com.codahale.metrics.MetricRegistry -trait Source { +private[spark] trait Source { def sourceName: String def metricRegistry: MetricRegistry } diff --git a/core/src/main/scala/org/apache/spark/metrics/source/package.scala b/core/src/main/scala/org/apache/spark/metrics/source/package.scala new file mode 100644 index 0000000000000..1e59fed6f7d7a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/metrics/source/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.metrics + +/** + * Sources used in Spark's metrics system. + */ +package object source diff --git a/core/src/main/scala/org/apache/spark/network/Connection.scala b/core/src/main/scala/org/apache/spark/network/Connection.scala index 2f7576c53b482..3b6298a26d7c5 100644 --- a/core/src/main/scala/org/apache/spark/network/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/Connection.scala @@ -210,7 +210,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, var nextMessageToBeUsed = 0 def addMessage(message: Message) { - messages.synchronized{ + messages.synchronized { /* messages += message */ messages.enqueue(message) logDebug("Added [" + message + "] to outbox for sending to " + @@ -223,7 +223,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, while (!messages.isEmpty) { /* nextMessageToBeUsed = nextMessageToBeUsed % messages.size */ /* val message = messages(nextMessageToBeUsed) */ - val message = messages.dequeue + val message = messages.dequeue() val chunk = message.getChunkForSending(defaultChunkSize) if (chunk.isDefined) { messages.enqueue(message) @@ -248,14 +248,14 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, } } - // outbox is used as a lock - ensure that it is always used as a leaf (since methods which + // outbox is used as a lock - ensure that it is always used as a leaf (since methods which // lock it are invoked in context of other locks) private val outbox = new Outbox() /* - This is orthogonal to whether we have pending bytes to write or not - and satisfies a slightly - different purpose. This flag is to see if we need to force reregister for write even when we + This is orthogonal to whether we have pending bytes to write or not - and satisfies a slightly + different purpose. This flag is to see if we need to force reregister for write even when we do not have any pending bytes to write to socket. - This can happen due to a race between adding pending buffers, and checking for existing of + This can happen due to a race between adding pending buffers, and checking for existing of data as detailed in https://github.com/mesos/spark/pull/791 */ private var needForceReregister = false diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionId.scala b/core/src/main/scala/org/apache/spark/network/ConnectionId.scala index ffaab677d411a..d579c165a1917 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionId.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionId.scala @@ -18,7 +18,7 @@ package org.apache.spark.network private[spark] case class ConnectionId(connectionManagerId: ConnectionManagerId, uniqId: Int) { - override def toString = connectionManagerId.host + "_" + connectionManagerId.port + "_" + uniqId + override def toString = connectionManagerId.host + "_" + connectionManagerId.port + "_" + uniqId } private[spark] object ConnectionId { @@ -26,9 +26,9 @@ private[spark] object ConnectionId { def createConnectionIdFromString(connectionIdString: String): ConnectionId = { val res = connectionIdString.split("_").map(_.trim()) if (res.size != 3) { - throw new Exception("Error converting ConnectionId string: " + connectionIdString + + throw new Exception("Error converting ConnectionId string: " + connectionIdString + " to a ConnectionId Object") } new ConnectionId(new ConnectionManagerId(res(0), res(1).toInt), res(2).toInt) - } + } } diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 6b0a972f0bbe0..cf1c985c2fff9 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -17,7 +17,6 @@ package org.apache.spark.network -import java.net._ import java.nio._ import java.nio.channels._ import java.nio.channels.spi._ @@ -34,6 +33,7 @@ import scala.collection.mutable.SynchronizedQueue import scala.concurrent.{Await, ExecutionContext, Future, Promise} import scala.concurrent.duration._ +import scala.language.postfixOps import org.apache.spark._ import org.apache.spark.util.{SystemClock, Utils} @@ -80,7 +80,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, private val serverChannel = ServerSocketChannel.open() // used to track the SendingConnections waiting to do SASL negotiation - private val connectionsAwaitingSasl = new HashMap[ConnectionId, SendingConnection] + private val connectionsAwaitingSasl = new HashMap[ConnectionId, SendingConnection] with SynchronizedMap[ConnectionId, SendingConnection] private val connectionsByKey = new HashMap[SelectionKey, Connection] with SynchronizedMap[SelectionKey, Connection] @@ -93,7 +93,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, implicit val futureExecContext = ExecutionContext.fromExecutor( Utils.newDaemonCachedThreadPool("Connection manager future execution context")) - private var onReceiveCallback: (BufferMessage, ConnectionManagerId) => Option[Message]= null + @volatile + private var onReceiveCallback: (BufferMessage, ConnectionManagerId) => Option[Message] = null private val authEnabled = securityManager.isAuthenticationEnabled() @@ -142,7 +143,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } finally { writeRunnableStarted.synchronized { writeRunnableStarted -= key - val needReregister = register || conn.resetForceReregister() + val needReregister = register || conn.resetForceReregister() if (needReregister && conn.changeInterestForWrite()) { conn.registerInterest() } @@ -249,14 +250,14 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, try { while(!selectorThread.isInterrupted) { while (! registerRequests.isEmpty) { - val conn: SendingConnection = registerRequests.dequeue + val conn: SendingConnection = registerRequests.dequeue() addListeners(conn) conn.connect() addConnection(conn) } while(!keyInterestChangeRequests.isEmpty) { - val (key, ops) = keyInterestChangeRequests.dequeue + val (key, ops) = keyInterestChangeRequests.dequeue() try { if (key.isValid) { @@ -510,7 +511,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, private def handleClientAuthentication( waitingConn: SendingConnection, - securityMsg: SecurityMessage, + securityMsg: SecurityMessage, connectionId : ConnectionId) { if (waitingConn.isSaslComplete()) { logDebug("Client sasl completed for id: " + waitingConn.connectionId) @@ -531,9 +532,9 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } return } - var securityMsgResp = SecurityMessage.fromResponse(replyToken, + val securityMsgResp = SecurityMessage.fromResponse(replyToken, securityMsg.getConnectionId.toString()) - var message = securityMsgResp.toBufferMessage + val message = securityMsgResp.toBufferMessage if (message == null) throw new Exception("Error creating security message") sendSecurityMessage(waitingConn.getRemoteConnectionManagerId(), message) } catch { @@ -547,7 +548,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } private def handleServerAuthentication( - connection: Connection, + connection: Connection, securityMsg: SecurityMessage, connectionId: ConnectionId) { if (!connection.isSaslComplete()) { @@ -562,17 +563,17 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } replyToken = connection.sparkSaslServer.response(securityMsg.getToken) if (connection.isSaslComplete()) { - logDebug("Server sasl completed: " + connection.connectionId) + logDebug("Server sasl completed: " + connection.connectionId) } else { logDebug("Server sasl not completed: " + connection.connectionId) } if (replyToken != null) { - var securityMsgResp = SecurityMessage.fromResponse(replyToken, + val securityMsgResp = SecurityMessage.fromResponse(replyToken, securityMsg.getConnectionId) - var message = securityMsgResp.toBufferMessage + val message = securityMsgResp.toBufferMessage if (message == null) throw new Exception("Error creating security Message") sendSecurityMessage(connection.getRemoteConnectionManagerId(), message) - } + } } catch { case e: Exception => { logError("Error in server auth negotiation: " + e) @@ -582,7 +583,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } } } else { - logDebug("connection already established for this connection id: " + connection.connectionId) + logDebug("connection already established for this connection id: " + connection.connectionId) } } @@ -610,14 +611,14 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, return true } else { if (!conn.isSaslComplete()) { - // We could handle this better and tell the client we need to do authentication - // negotiation, but for now just ignore them. + // We could handle this better and tell the client we need to do authentication + // negotiation, but for now just ignore them. logError("message sent that is not security negotiation message on connection " + "not authenticated yet, ignoring it!!") return true } } - return false + false } private def handleMessage( @@ -693,9 +694,9 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, var firstResponse: Array[Byte] = null try { firstResponse = conn.sparkSaslClient.firstToken() - var securityMsg = SecurityMessage.fromResponse(firstResponse, + val securityMsg = SecurityMessage.fromResponse(firstResponse, conn.connectionId.toString()) - var message = securityMsg.toBufferMessage + val message = securityMsg.toBufferMessage if (message == null) throw new Exception("Error creating security message") connectionsAwaitingSasl += ((conn.connectionId, conn)) sendSecurityMessage(connManagerId, message) @@ -710,11 +711,11 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } } } else { - logDebug("Sasl already established ") + logDebug("Sasl already established ") } } - // allow us to add messages to the inbox for doing sasl negotiating + // allow us to add messages to the inbox for doing sasl negotiating private def sendSecurityMessage(connManagerId: ConnectionManagerId, message: Message) { def startNewConnection(): SendingConnection = { val inetSocketAddress = new InetSocketAddress(connManagerId.host, connManagerId.port) @@ -773,7 +774,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, if (((clock.getTime() - startTime) >= (authTimeout * 1000)) && (!connection.isSaslComplete())) { // took to long to authenticate the connection, something probably went wrong - throw new Exception("Took to long for authentication to " + connectionManagerId + + throw new Exception("Took to long for authentication to " + connectionManagerId + ", waited " + authTimeout + "seconds, failing.") } } @@ -795,7 +796,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } } case None => { - logError("no messageStatus for failed message id: " + message.id) + logError("no messageStatus for failed message id: " + message.id) } } } diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala index 9d9b9dbdd5331..4894ecd41f6eb 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala @@ -37,11 +37,11 @@ private[spark] object ConnectionManagerTest extends Logging{ "[size of msg in MB (integer)] [count] [await time in seconds)] ") System.exit(1) } - + if (args(0).startsWith("local")) { println("This runs only on a mesos cluster") } - + val sc = new SparkContext(args(0), "ConnectionManagerTest") val slavesFile = Source.fromFile(args(1)) val slaves = slavesFile.mkString.split("\n") @@ -50,7 +50,7 @@ private[spark] object ConnectionManagerTest extends Logging{ /* println("Slaves") */ /* slaves.foreach(println) */ val tasknum = if (args.length > 2) args(2).toInt else slaves.length - val size = ( if (args.length > 3) (args(3).toInt) else 10 ) * 1024 * 1024 + val size = ( if (args.length > 3) (args(3).toInt) else 10 ) * 1024 * 1024 val count = if (args.length > 4) args(4).toInt else 3 val awaitTime = (if (args.length > 5) args(5).toInt else 600 ).second println("Running " + count + " rounds of test: " + "parallel tasks = " + tasknum + ", " + @@ -64,16 +64,16 @@ private[spark] object ConnectionManagerTest extends Logging{ (0 until count).foreach(i => { val resultStrs = sc.parallelize(0 until tasknum, tasknum).map(i => { val connManager = SparkEnv.get.connectionManager - val thisConnManagerId = connManager.id - connManager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + val thisConnManagerId = connManager.id + connManager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { logInfo("Received [" + msg + "] from [" + id + "]") None }) val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) buffer.flip - - val startTime = System.currentTimeMillis + + val startTime = System.currentTimeMillis val futures = slaveConnManagerIds.filter(_ != thisConnManagerId).map{ slaveConnManagerId => { val bufferMessage = Message.createBufferMessage(buffer.duplicate) @@ -84,7 +84,7 @@ private[spark] object ConnectionManagerTest extends Logging{ val results = futures.map(f => Await.result(f, awaitTime)) val finishTime = System.currentTimeMillis Thread.sleep(5000) - + val mb = size * results.size / 1024.0 / 1024.0 val ms = finishTime - startTime val resultStr = thisConnManagerId + " Sent " + mb + " MB in " + ms + " ms at " + (mb / ms * @@ -92,11 +92,11 @@ private[spark] object ConnectionManagerTest extends Logging{ logInfo(resultStr) resultStr }).collect() - - println("---------------------") - println("Run " + i) + + println("---------------------") + println("Run " + i) resultStrs.foreach(println) - println("---------------------") + println("---------------------") }) } } diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala index 2b41c403b2e0a..53a6038a9b59e 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala @@ -18,7 +18,7 @@ package org.apache.spark.network import java.nio.ByteBuffer -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} private[spark] object ReceiverTest { def main(args: Array[String]) { @@ -28,7 +28,7 @@ private[spark] object ReceiverTest { manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { /* println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis) */ - val buffer = ByteBuffer.wrap("response".getBytes) + val buffer = ByteBuffer.wrap("response".getBytes("utf-8")) Some(Message.createBufferMessage(buffer, msg.id)) }) Thread.currentThread.join() diff --git a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala b/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala index 0d9f743b3624b..9af9e2e8e9e59 100644 --- a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala @@ -26,33 +26,33 @@ import org.apache.spark._ import org.apache.spark.network._ /** - * SecurityMessage is class that contains the connectionId and sasl token + * SecurityMessage is class that contains the connectionId and sasl token * used in SASL negotiation. SecurityMessage has routines for converting * it to and from a BufferMessage so that it can be sent by the ConnectionManager * and easily consumed by users when received. * The api was modeled after BlockMessage. * - * The connectionId is the connectionId of the client side. Since + * The connectionId is the connectionId of the client side. Since * message passing is asynchronous and its possible for the server side (receiving) - * to get multiple different types of messages on the same connection the connectionId - * is used to know which connnection the security message is intended for. - * + * to get multiple different types of messages on the same connection the connectionId + * is used to know which connnection the security message is intended for. + * * For instance, lets say we are node_0. We need to send data to node_1. The node_0 side * is acting as a client and connecting to node_1. SASL negotiation has to occur - * between node_0 and node_1 before node_1 trusts node_0 so node_0 sends a security message. - * node_1 receives the message from node_0 but before it can process it and send a response, - * some thread on node_1 decides it needs to send data to node_0 so it connects to node_0 - * and sends a security message of its own to authenticate as a client. Now node_0 gets - * the message and it needs to decide if this message is in response to it being a client - * (from the first send) or if its just node_1 trying to connect to it to send data. This + * between node_0 and node_1 before node_1 trusts node_0 so node_0 sends a security message. + * node_1 receives the message from node_0 but before it can process it and send a response, + * some thread on node_1 decides it needs to send data to node_0 so it connects to node_0 + * and sends a security message of its own to authenticate as a client. Now node_0 gets + * the message and it needs to decide if this message is in response to it being a client + * (from the first send) or if its just node_1 trying to connect to it to send data. This * is where the connectionId field is used. node_0 can lookup the connectionId to see if * it is in response to it being a client or if its in response to someone sending other data. - * + * * The format of a SecurityMessage as its sent is: * - Length of the ConnectionId - * - ConnectionId + * - ConnectionId * - Length of the token - * - Token + * - Token */ private[spark] class SecurityMessage() extends Logging { @@ -61,13 +61,13 @@ private[spark] class SecurityMessage() extends Logging { def set(byteArr: Array[Byte], newconnectionId: String) { if (byteArr == null) { - token = new Array[Byte](0) + token = new Array[Byte](0) } else { token = byteArr } connectionId = newconnectionId } - + /** * Read the given buffer and set the members of this class. */ @@ -91,31 +91,30 @@ private[spark] class SecurityMessage() extends Logging { buffer.clear() set(buffer) } - + def getConnectionId: String = { return connectionId } - + def getToken: Array[Byte] = { return token } - + /** - * Create a BufferMessage that can be sent by the ConnectionManager containing + * Create a BufferMessage that can be sent by the ConnectionManager containing * the security information from this class. * @return BufferMessage */ def toBufferMessage: BufferMessage = { - val startTime = System.currentTimeMillis val buffers = new ArrayBuffer[ByteBuffer]() // 4 bytes for the length of the connectionId - // connectionId is of type char so multiple the length by 2 to get number of bytes + // connectionId is of type char so multiple the length by 2 to get number of bytes // 4 bytes for the length of token // token is a byte buffer so just take the length var buffer = ByteBuffer.allocate(4 + connectionId.length() * 2 + 4 + token.length) buffer.putInt(connectionId.length()) - connectionId.foreach((x: Char) => buffer.putChar(x)) + connectionId.foreach((x: Char) => buffer.putChar(x)) buffer.putInt(token.length) if (token.length > 0) { @@ -123,7 +122,7 @@ private[spark] class SecurityMessage() extends Logging { } buffer.flip() buffers += buffer - + var message = Message.createBufferMessage(buffers) logDebug("message total size is : " + message.size) message.isSecurityNeg = true @@ -136,7 +135,7 @@ private[spark] class SecurityMessage() extends Logging { } private[spark] object SecurityMessage { - + /** * Convert the given BufferMessage to a SecurityMessage by parsing the contents * of the BufferMessage and populating the SecurityMessage fields. diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala index 14c094c6177d5..b8ea7c2cff9a2 100644 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala @@ -54,7 +54,7 @@ private[spark] object SenderTest { val responseStr = manager.sendMessageReliablySync(targetConnectionManagerId, dataMessage) .map { response => val buffer = response.asInstanceOf[BufferMessage].buffers(0) - new String(buffer.array) + new String(buffer.array, "utf-8") }.getOrElse("none") val finishTime = System.currentTimeMillis diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala index 4164e81d3a8ae..136c1912045aa 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala @@ -36,8 +36,8 @@ private[spark] class FileHeader ( if (FileHeader.HEADER_SIZE - buf.readableBytes > 0 ) { buf.writeZero(FileHeader.HEADER_SIZE - buf.readableBytes) } else { - throw new Exception("too long header " + buf.readableBytes) - logInfo("too long header") + throw new Exception("too long header " + buf.readableBytes) + logInfo("too long header") } buf } diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 2625a7f6a575a..5cdbc306e56a0 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -30,9 +30,18 @@ package org.apache * type (e.g. RDD[(Int, Int)] through implicit conversions when you * `import org.apache.spark.SparkContext._`. * - * Java programmers should reference the [[spark.api.java]] package + * Java programmers should reference the [[org.apache.spark.api.java]] package * for Spark programming APIs in Java. + * + * Classes and methods marked with + * Experimental are user-facing features which have not been officially adopted by the + * Spark project. These are subject to change or removal in minor releases. + * + * Classes and methods marked with + * Developer API are intended for advanced users want to extend Spark through lower + * level interfaces. These are subject to changes or removal in minor releases. */ + package object spark { // For package docs only } diff --git a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala index 5f4450859cc9b..aed0353344427 100644 --- a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala @@ -17,9 +17,13 @@ package org.apache.spark.partial +import org.apache.spark.annotation.Experimental + /** - * A Double with error bars on it. + * :: Experimental :: + * A Double value with error bars and associated confidence. */ +@Experimental class BoundedDouble(val mean: Double, val confidence: Double, val low: Double, val high: Double) { override def toString(): String = "[%.3f, %.3f]".format(low, high) } diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala index 40b70baabcad9..8bb78123e3c9c 100644 --- a/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala @@ -22,36 +22,33 @@ import java.util.{HashMap => JHashMap} import scala.collection.JavaConversions.mapAsScalaMap import scala.collection.Map import scala.collection.mutable.HashMap +import scala.reflect.ClassTag import cern.jet.stat.Probability -import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} + +import org.apache.spark.util.collection.OpenHashMap /** * An ApproximateEvaluator for counts by key. Returns a map of key to confidence interval. */ -private[spark] class GroupedCountEvaluator[T](totalOutputs: Int, confidence: Double) - extends ApproximateEvaluator[OLMap[T], Map[T, BoundedDouble]] { +private[spark] class GroupedCountEvaluator[T : ClassTag](totalOutputs: Int, confidence: Double) + extends ApproximateEvaluator[OpenHashMap[T,Long], Map[T, BoundedDouble]] { var outputsMerged = 0 - var sums = new OLMap[T] // Sum of counts for each key + var sums = new OpenHashMap[T,Long]() // Sum of counts for each key - override def merge(outputId: Int, taskResult: OLMap[T]) { + override def merge(outputId: Int, taskResult: OpenHashMap[T,Long]) { outputsMerged += 1 - val iter = taskResult.object2LongEntrySet.fastIterator() - while (iter.hasNext) { - val entry = iter.next() - sums.put(entry.getKey, sums.getLong(entry.getKey) + entry.getLongValue) + taskResult.foreach { case (key, value) => + sums.changeValue(key, value, _ + value) } } override def currentResult(): Map[T, BoundedDouble] = { if (outputsMerged == totalOutputs) { val result = new JHashMap[T, BoundedDouble](sums.size) - val iter = sums.object2LongEntrySet.fastIterator() - while (iter.hasNext) { - val entry = iter.next() - val sum = entry.getLongValue() - result(entry.getKey) = new BoundedDouble(sum, 1.0, sum, sum) + sums.foreach { case (key, sum) => + result(key) = new BoundedDouble(sum, 1.0, sum, sum) } result } else if (outputsMerged == 0) { @@ -60,16 +57,13 @@ private[spark] class GroupedCountEvaluator[T](totalOutputs: Int, confidence: Dou val p = outputsMerged.toDouble / totalOutputs val confFactor = Probability.normalInverse(1 - (1 - confidence) / 2) val result = new JHashMap[T, BoundedDouble](sums.size) - val iter = sums.object2LongEntrySet.fastIterator() - while (iter.hasNext) { - val entry = iter.next() - val sum = entry.getLongValue + sums.foreach { case (key, sum) => val mean = (sum + 1 - p) / p val variance = (sum + 1) * (1 - p) / (p * p) val stdev = math.sqrt(variance) val low = mean - confFactor * stdev val high = mean + confFactor * stdev - result(entry.getKey) = new BoundedDouble(mean, confidence, low, high) + result(key) = new BoundedDouble(mean, confidence, low, high) } result } diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala index b5111891ed35b..af26c3d59ac02 100644 --- a/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala @@ -61,7 +61,6 @@ private[spark] class GroupedMeanEvaluator[T](totalOutputs: Int, confidence: Doub } else if (outputsMerged == 0) { new HashMap[T, BoundedDouble] } else { - val p = outputsMerged.toDouble / totalOutputs val studentTCacher = new StudentTCacher(confidence) val result = new JHashMap[T, BoundedDouble](sums.size) val iter = sums.entrySet.iterator() 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 812368e04ac0d..cadd0c7ed19ba 100644 --- a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala +++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala @@ -17,6 +17,9 @@ package org.apache.spark.partial +import org.apache.spark.annotation.Experimental + +@Experimental class PartialResult[R](initialVal: R, isFinal: Boolean) { private var finalValue: Option[R] = if (isFinal) Some(initialVal) else None private var failure: Option[Exception] = None @@ -41,7 +44,7 @@ class PartialResult[R](initialVal: R, isFinal: Boolean) { } } - /** + /** * Set a handler to be called when this PartialResult completes. Only one completion handler * is supported per PartialResult. */ @@ -57,7 +60,7 @@ class PartialResult[R](initialVal: R, isFinal: Boolean) { return this } - /** + /** * Set a handler to be called if this PartialResult's job fails. Only one failure handler * is supported per PartialResult. */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala b/core/src/main/scala/org/apache/spark/partial/package.scala similarity index 74% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala rename to core/src/main/scala/org/apache/spark/partial/package.scala index e4ef3c58e8680..62dc5cd25a164 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala +++ b/core/src/main/scala/org/apache/spark/partial/package.scala @@ -15,16 +15,14 @@ * limitations under the License. */ -package org.apache.spark.mllib.linalg - -import org.apache.spark.rdd.RDD - +package org.apache.spark /** - * Class that represents a dense matrix + * :: Experimental :: + * + * Support for approximate results. This provides convenient api and also implementation for + * approximate calculation. * - * @param rows RDD of rows - * @param m number of rows - * @param n number of columns + * @see [[org.apache.spark.rdd.RDD.countApprox]] */ -case class TallSkinnyDenseMatrix(val rows: RDD[MatrixRow], val m: Int, val n: Int) +package object partial 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 d1c74a5063510..aed951a40b40c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -24,11 +24,14 @@ import scala.concurrent.ExecutionContext.Implicits.global import scala.reflect.ClassTag import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} +import org.apache.spark.annotation.Experimental /** + * :: Experimental :: * A set of asynchronous RDD actions available through an implicit conversion. * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ +@Experimental class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Logging { /** 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 e6c4a6d3794a0..c64da8804d166 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -19,24 +19,30 @@ package org.apache.spark.rdd import scala.reflect.ClassTag -import org.apache.spark.{Partition, SparkContext, SparkEnv, TaskContext} +import org.apache.spark._ import org.apache.spark.storage.{BlockId, BlockManager} +import scala.Some private[spark] class BlockRDDPartition(val blockId: BlockId, idx: Int) extends Partition { val index = idx } private[spark] -class BlockRDD[T: ClassTag](sc: SparkContext, @transient blockIds: Array[BlockId]) +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) + @volatile private var _isValid = true - override def getPartitions: Array[Partition] = (0 until blockIds.size).map(i => { - new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition] - }).toArray + override def getPartitions: Array[Partition] = { + assertValid() + (0 until blockIds.size).map(i => { + new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition] + }).toArray + } override def compute(split: Partition, context: TaskContext): Iterator[T] = { + assertValid() val blockManager = SparkEnv.get.blockManager val blockId = split.asInstanceOf[BlockRDDPartition].blockId blockManager.get(blockId) match { @@ -47,7 +53,36 @@ class BlockRDD[T: ClassTag](sc: SparkContext, @transient blockIds: Array[BlockId } override def getPreferredLocations(split: Partition): Seq[String] = { + assertValid() locations_(split.asInstanceOf[BlockRDDPartition].blockId) } + + /** + * Remove the data blocks that this BlockRDD is made from. NOTE: This is an + * irreversible operation, as the data in the blocks cannot be recovered back + * once removed. Use it with caution. + */ + private[spark] def removeBlocks() { + blockIds.foreach { blockId => + sc.env.blockManager.master.removeBlock(blockId) + } + _isValid = false + } + + /** + * Whether this BlockRDD is actually usable. This will be false if the data blocks have been + * removed using `this.removeBlocks`. + */ + private[spark] def isValid: Boolean = { + _isValid + } + + /** Check if this BlockRDD is valid. If not valid, exception is thrown. */ + private[spark] def assertValid() { + if (!_isValid) { + throw new SparkException( + "Attempted to use %s after its blocks have been removed!".format(toString)) + } + } } 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 888af541cf970..34c51b833025e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -84,7 +84,7 @@ private[spark] object CheckpointRDD extends Logging { "part-%05d".format(splitId) } - def writeToFile[T]( + def writeToFile[T: ClassTag]( path: String, broadcastedConf: Broadcast[SerializableWritable[Configuration]], blockSize: Int = -1 @@ -160,7 +160,7 @@ private[spark] object CheckpointRDD extends Logging { val conf = SparkHadoopUtil.get.newConfiguration() val fs = path.getFileSystem(conf) val broadcastedConf = sc.broadcast(new SerializableWritable(conf)) - sc.runJob(rdd, CheckpointRDD.writeToFile(path.toString, broadcastedConf, 1024) _) + sc.runJob(rdd, CheckpointRDD.writeToFile[Int](path.toString, broadcastedConf, 1024) _) val cpRDD = new CheckpointRDD[Int](sc, path.toString) assert(cpRDD.partitions.length == rdd.partitions.length, "Number of partitions is not the same") assert(cpRDD.collect.toList == rdd.collect.toList, "Data of partitions not the same") 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 9aa454a5c8b88..5951865e56c9d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -20,11 +20,14 @@ package org.apache.spark.rdd import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable.ArrayBuffer +import scala.language.existentials import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.ShuffleHandle private[spark] sealed trait CoGroupSplitDep extends Serializable @@ -42,7 +45,7 @@ private[spark] case class NarrowCoGroupSplitDep( } } -private[spark] case class ShuffleCoGroupSplitDep(shuffleId: Int) extends CoGroupSplitDep +private[spark] case class ShuffleCoGroupSplitDep(handle: ShuffleHandle) extends CoGroupSplitDep private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep]) extends Partition with Serializable { @@ -51,12 +54,17 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] } /** + * :: DeveloperApi :: * A RDD that cogroups its parents. For each key k in parent RDDs, the resulting RDD contains a * tuple with the list of values for that key. * + * Note: This is an internal API. We recommend users use RDD.coGroup(...) instead of + * instantiating this directly. + * @param rdds parent RDDs. - * @param part partitioner used to partition the shuffle output. + * @param part partitioner used to partition the shuffle output */ +@DeveloperApi class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner) extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { @@ -67,10 +75,11 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: private type CoGroupValue = (Any, Int) // Int is dependency number private type CoGroupCombiner = Seq[CoGroup] - private var serializer: Serializer = null + private var serializer: Option[Serializer] = None + /** Set a serializer for this RDD's shuffle, or null to use the default (spark.serializer) */ def setSerializer(serializer: Serializer): CoGroupedRDD[K] = { - this.serializer = serializer + this.serializer = Option(serializer) this } @@ -81,7 +90,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: new OneToOneDependency(rdd) } else { logDebug("Adding shuffle dependency with " + rdd) - new ShuffleDependency[Any, Any](rdd, part, serializer) + new ShuffleDependency[K, Any, CoGroupCombiner](rdd, part, serializer) } } } @@ -93,8 +102,8 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: 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.shuffleId) + case s: ShuffleDependency[_, _, _] => + new ShuffleCoGroupSplitDep(s.shuffleHandle) case _ => new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i)) } @@ -119,11 +128,11 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: val it = rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]] rddIterators += ((it, depNum)) - case ShuffleCoGroupSplitDep(shuffleId) => + case ShuffleCoGroupSplitDep(handle) => // Read map outputs of shuffle - val fetcher = SparkEnv.get.shuffleFetcher - val ser = Serializer.getSerializer(serializer) - val it = fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser) + val it = SparkEnv.get.shuffleManager + .getReader(handle, 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 4e82b51313bf0..c45b759f007cc 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -21,6 +21,7 @@ import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.language.existentials import scala.reflect.ClassTag import org.apache.spark._ @@ -32,7 +33,7 @@ import org.apache.spark._ * @param parentsIndices list of indices in the parent that have been coalesced into this partition * @param preferredLocation the preferred location for this partition */ -case class CoalescedRDDPartition( +private[spark] case class CoalescedRDDPartition( index: Int, @transient rdd: RDD[_], parentsIndices: Array[Int], @@ -70,7 +71,7 @@ case class CoalescedRDDPartition( * @param maxPartitions number of desired partitions in the coalesced RDD * @param balanceSlack used to trade-off balance and locality. 1.0 is all locality, 0 is all balance */ -class CoalescedRDD[T: ClassTag]( +private[spark] class CoalescedRDD[T: ClassTag]( @transient var prev: RDD[T], maxPartitions: Int, balanceSlack: Double = 0.10) 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 a7b6b3b5146ce..9ca971c8a4c27 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -17,6 +17,7 @@ package org.apache.spark.rdd +import org.apache.spark.annotation.Experimental import org.apache.spark.{TaskContext, Logging} import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.MeanEvaluator @@ -51,7 +52,7 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { /** Compute the standard deviation of this RDD's elements. */ def stdev(): Double = 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). */ @@ -63,14 +64,22 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { */ def sampleVariance(): Double = stats().sampleVariance - /** (Experimental) Approximate operation to return the mean within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the mean within a timeout. + */ + @Experimental def meanApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns) val evaluator = new MeanEvaluator(self.partitions.size, confidence) self.context.runApproximateJob(self, processPartition, evaluator, timeout) } - /** (Experimental) Approximate operation to return the sum within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the sum within a timeout. + */ + @Experimental def sumApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns) val evaluator = new SumEvaluator(self.partitions.size, confidence) @@ -114,13 +123,13 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { * e.g. for the array * [1, 10, 20, 50] the buckets are [1, 10) [10, 20) [20, 50] * e.g 1<=x<10 , 10<=x<20, 20<=x<50 - * And on the input of 1 and 50 we would have a histogram of 1, 0, 0 - * + * And on the input of 1 and 50 we would have a histogram of 1, 0, 0 + * * Note: if your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched * from an O(log n) inseration to O(1) per element. (where n = # buckets) if you set evenBuckets * to true. * buckets must be sorted and not contain any duplicates. - * buckets array must be at least two elements + * buckets array must be at least two elements * All NaN entries are treated the same. If you have a NaN bucket it must be * the maximum value of the last position and all NaN entries will be counted * in that bucket. diff --git a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala index a84e5f9fd8ef8..a2d7e344cf1b2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala @@ -22,9 +22,9 @@ import scala.reflect.ClassTag import org.apache.spark.{Partition, SparkContext, TaskContext} /** - * An RDD that is empty, i.e. has no element in it. + * An RDD that has no partitions and no elements. */ -class EmptyRDD[T: ClassTag](sc: SparkContext) extends RDD[T](sc, Nil) { +private[spark] class EmptyRDD[T: ClassTag](sc: SparkContext) extends RDD[T](sc, Nil) { override def getPartitions: Array[Partition] = Array.empty 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 3af008bd72378..2aa111d600e9b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.mapred.TaskID import org.apache.hadoop.util.ReflectionUtils import org.apache.spark._ +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.NextIterator @@ -70,9 +71,13 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp } /** + * :: DeveloperApi :: * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, * sources in HBase, or S3), using the older MapReduce API (`org.apache.hadoop.mapred`). * + * Note: Instantiating this class directly is not recommended, please use + * [[org.apache.spark.SparkContext.hadoopRDD()]] + * * @param sc The SparkContext to associate the RDD with. * @param broadcastedConf A general Hadoop Configuration, or a subclass of it. If the enclosed * variabe references an instance of JobConf, then that JobConf will be used for the Hadoop job. @@ -82,8 +87,9 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp * @param inputFormatClass Storage format of the data to be read. * @param keyClass Class of the key associated with the inputFormatClass. * @param valueClass Class of the value associated with the inputFormatClass. - * @param minSplits Minimum number of Hadoop Splits (HadoopRDD partitions) to generate. + * @param minPartitions Minimum number of HadoopRDD partitions (Hadoop Splits) to generate. */ +@DeveloperApi class HadoopRDD[K, V]( sc: SparkContext, broadcastedConf: Broadcast[SerializableWritable[Configuration]], @@ -91,7 +97,7 @@ class HadoopRDD[K, V]( inputFormatClass: Class[_ <: InputFormat[K, V]], keyClass: Class[K], valueClass: Class[V], - minSplits: Int) + minPartitions: Int) extends RDD[(K, V)](sc, Nil) with Logging { def this( @@ -100,7 +106,7 @@ class HadoopRDD[K, V]( inputFormatClass: Class[_ <: InputFormat[K, V]], keyClass: Class[K], valueClass: Class[V], - minSplits: Int) = { + minPartitions: Int) = { this( sc, sc.broadcast(new SerializableWritable(conf)) @@ -109,7 +115,7 @@ class HadoopRDD[K, V]( inputFormatClass, keyClass, valueClass, - minSplits) + minPartitions) } protected val jobConfCacheKey = "rdd_%d_job_conf".format(id) @@ -133,10 +139,13 @@ class HadoopRDD[K, V]( // Create a JobConf that will be cached and used across this RDD's getJobConf() calls in the // local process. The local cache is accessed through HadoopRDD.putCachedMetadata(). // The caching helps minimize GC, since a JobConf can contain ~10KB of temporary objects. - val newJobConf = new JobConf(broadcastedConf.value.value) - initLocalJobConfFuncOpt.map(f => f(newJobConf)) - HadoopRDD.putCachedMetadata(jobConfCacheKey, newJobConf) - newJobConf + // synchronize to prevent ConcurrentModificationException (Spark-1097, Hadoop-10456) + broadcastedConf.synchronized { + val newJobConf = new JobConf(broadcastedConf.value.value) + initLocalJobConfFuncOpt.map(f => f(newJobConf)) + HadoopRDD.putCachedMetadata(jobConfCacheKey, newJobConf) + newJobConf + } } } @@ -163,7 +172,7 @@ class HadoopRDD[K, V]( if (inputFormat.isInstanceOf[Configurable]) { inputFormat.asInstanceOf[Configurable].setConf(jobConf) } - val inputSplits = inputFormat.getSplits(jobConf, minSplits) + val inputSplits = inputFormat.getSplits(jobConf, minPartitions) val array = new Array[Partition](inputSplits.size) for (i <- 0 until inputSplits.size) { array(i) = new HadoopPartition(id, i, inputSplits(i)) 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 1b503743ac117..a76a070b5b863 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.NextIterator private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { override def index = 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. * For usage example, see test case JdbcRDDSuite. 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 461a749eac48b..ac1ccc06f238a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -24,10 +24,18 @@ import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ -import org.apache.spark.{InterruptibleIterator, Logging, Partition, SerializableWritable, SparkContext, TaskContext} - -private[spark] -class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable) +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.input.WholeTextFileInputFormat +import org.apache.spark.InterruptibleIterator +import org.apache.spark.Logging +import org.apache.spark.Partition +import org.apache.spark.SerializableWritable +import org.apache.spark.{SparkContext, TaskContext} + +private[spark] class NewHadoopPartition( + rddId: Int, + val index: Int, + @transient rawSplit: InputSplit with Writable) extends Partition { val serializableHadoopSplit = new SerializableWritable(rawSplit) @@ -36,15 +44,20 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS } /** + * :: DeveloperApi :: * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, * sources in HBase, or S3), using the new MapReduce API (`org.apache.hadoop.mapreduce`). * + * Note: Instantiating this class directly is not recommended, please use + * [[org.apache.spark.SparkContext.newAPIHadoopRDD()]] + * * @param sc The SparkContext to associate the RDD with. * @param inputFormatClass Storage format of the data to be read. * @param keyClass Class of the key associated with the inputFormatClass. * @param valueClass Class of the value associated with the inputFormatClass. * @param conf The Hadoop configuration. */ +@DeveloperApi class NewHadoopRDD[K, V]( sc : SparkContext, inputFormatClass: Class[_ <: InputFormat[K, V]], @@ -59,17 +72,19 @@ class NewHadoopRDD[K, V]( private val confBroadcast = sc.broadcast(new SerializableWritable(conf)) // private val serializableConf = new SerializableWritable(conf) - private val jobtrackerId: String = { + private val jobTrackerId: String = { val formatter = new SimpleDateFormat("yyyyMMddHHmm") formatter.format(new Date()) } - @transient private val jobId = new JobID(jobtrackerId, id) + @transient protected val jobId = new JobID(jobTrackerId, id) override def getPartitions: Array[Partition] = { val inputFormat = inputFormatClass.newInstance - if (inputFormat.isInstanceOf[Configurable]) { - inputFormat.asInstanceOf[Configurable].setConf(conf) + inputFormat match { + case configurable: Configurable => + configurable.setConf(conf) + case _ => } val jobContext = newJobContext(conf, jobId) val rawSplits = inputFormat.getSplits(jobContext).toArray @@ -85,11 +100,13 @@ class NewHadoopRDD[K, V]( val split = theSplit.asInstanceOf[NewHadoopPartition] logInfo("Input split: " + split.serializableHadoopSplit) val conf = confBroadcast.value.value - val attemptId = newTaskAttemptID(jobtrackerId, id, isMap = true, split.index, 0) + val attemptId = newTaskAttemptID(jobTrackerId, id, isMap = true, split.index, 0) val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) val format = inputFormatClass.newInstance - if (format.isInstanceOf[Configurable]) { - format.asInstanceOf[Configurable].setConf(conf) + format match { + case configurable: Configurable => + configurable.setConf(conf) + case _ => } val reader = format.createRecordReader( split.serializableHadoopSplit.value, hadoopAttemptContext) @@ -135,3 +152,30 @@ class NewHadoopRDD[K, V]( def getConf: Configuration = confBroadcast.value.value } +private[spark] class WholeTextFileRDD( + sc : SparkContext, + inputFormatClass: Class[_ <: WholeTextFileInputFormat], + keyClass: Class[String], + valueClass: Class[String], + @transient conf: Configuration, + minPartitions: Int) + extends NewHadoopRDD[String, String](sc, inputFormatClass, keyClass, valueClass, conf) { + + override def getPartitions: Array[Partition] = { + val inputFormat = inputFormatClass.newInstance + inputFormat match { + case configurable: Configurable => + configurable.setConf(conf) + case _ => + } + val jobContext = newJobContext(conf, jobId) + inputFormat.setMaxSplitSize(jobContext, minPartitions) + val rawSplits = inputFormat.getSplits(jobContext).toArray + val result = new Array[Partition](rawSplits.size) + for (i <- 0 until rawSplits.size) { + result(i) = new NewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) + } + result + } +} + 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 d5691f2267bfa..6a3f698444283 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -24,15 +24,31 @@ import org.apache.spark.{Logging, RangePartitioner} /** * Extra functions available on RDDs of (key, value) pairs where the key is sortable through * an implicit conversion. Import `org.apache.spark.SparkContext._` at the top of your program to - * use these functions. They will work with any key type that has a `scala.math.Ordered` - * implementation. + * use these functions. They will work with any key type `K` that has an implicit `Ordering[K]` in + * scope. Ordering objects already exist for all of the standard primitive types. Users can also + * define their own orderings for custom types, or to override the default ordering. The implicit + * ordering that is in the closest scope will be used. + * + * {{{ + * import org.apache.spark.SparkContext._ + * + * val rdd: RDD[(String, Int)] = ... + * implicit val caseInsensitiveOrdering = new Ordering[String] { + * override def compare(a: String, b: String) = a.toLowerCase.compare(b.toLowerCase) + * } + * + * // Sort by key, using the above case insensitive ordering. + * rdd.sortByKey() + * }}} */ -class OrderedRDDFunctions[K <% Ordered[K]: ClassTag, +class OrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag, P <: Product2[K, V] : ClassTag]( self: RDD[P]) extends Logging with Serializable { + private val ordering = implicitly[Ordering[K]] + /** * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling * `collect` or `save` on the resulting RDD will return or output an ordered list of records @@ -45,9 +61,9 @@ class OrderedRDDFunctions[K <% Ordered[K]: ClassTag, shuffled.mapPartitions(iter => { val buf = iter.toArray if (ascending) { - buf.sortWith((x, y) => x._1 < y._1).iterator + buf.sortWith((x, y) => ordering.lt(x._1, y._1)).iterator } else { - buf.sortWith((x, y) => x._1 > y._1).iterator + buf.sortWith((x, y) => ordering.gt(x._1, y._1)).iterator } }, preservesPartitioning = true) } 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 14386ff5b9127..443d1c587c3ee 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -28,7 +28,7 @@ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag -import com.clearspring.analytics.stream.cardinality.HyperLogLog +import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.io.SequenceFile.CompressionType @@ -39,23 +39,24 @@ RecordWriter => NewRecordWriter, SparkHadoopMapReduceUtil} import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} import org.apache.spark._ +import org.apache.spark.annotation.Experimental import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.SparkHadoopWriter import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.SparkContext._ import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer -import org.apache.spark.util.SerializableHyperLogLog /** * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ -class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) +class PairRDDFunctions[K, V](self: RDD[(K, V)]) + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) extends Logging with SparkHadoopMapReduceUtil - with Serializable { - + with Serializable +{ /** * 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 @@ -76,7 +77,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) mapSideCombine: Boolean = true, serializer: Serializer = null): RDD[(K, C)] = { require(mergeCombiners != null, "mergeCombiners must be defined") // required as of Spark 0.9.0 - if (getKeyClass().isArray) { + if (keyClass.isArray) { if (mapSideCombine) { throw new SparkException("Cannot use map-side combining with array keys.") } @@ -117,6 +118,56 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions)) } + /** + * Aggregate the values of each key, 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, + * as in scala.TraversableOnce. 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. + */ + def aggregateByKey[U: ClassTag](zeroValue: U, partitioner: Partitioner)(seqOp: (U, V) => U, + combOp: (U, U) => U): RDD[(K, U)] = { + // 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.closureSerializer.newInstance().serialize(zeroValue) + val zeroArray = new Array[Byte](zeroBuffer.limit) + zeroBuffer.get(zeroArray) + + lazy val cachedSerializer = SparkEnv.get.closureSerializer.newInstance() + def createZero() = cachedSerializer.deserialize[U](ByteBuffer.wrap(zeroArray)) + + combineByKey[U]((v: V) => seqOp(createZero(), v), seqOp, combOp, partitioner) + } + + /** + * Aggregate the values of each key, 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, + * as in scala.TraversableOnce. 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. + */ + def aggregateByKey[U: ClassTag](zeroValue: U, numPartitions: Int)(seqOp: (U, V) => U, + combOp: (U, U) => U): RDD[(K, U)] = { + aggregateByKey(zeroValue, new HashPartitioner(numPartitions))(seqOp, combOp) + } + + /** + * Aggregate the values of each key, 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, + * as in scala.TraversableOnce. 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. + */ + def aggregateByKey[U: ClassTag](zeroValue: U)(seqOp: (U, V) => U, + combOp: (U, U) => U): RDD[(K, U)] = { + aggregateByKey(zeroValue, defaultPartitioner(self))(seqOp, combOp) + } + /** * Merge the values for each key using an associative function and a neutral "zero value" which * may be added to the result an arbitrary number of times, and must not change the result @@ -169,7 +220,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def reduceByKeyLocally(func: (V, V) => V): Map[K, V] = { - if (getKeyClass().isArray) { + if (keyClass.isArray) { throw new SparkException("reduceByKeyLocally() does not support array keys") } @@ -201,48 +252,99 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def countByKey(): Map[K, Long] = self.map(_._1).countByValue() /** - * (Experimental) Approximate version of countByKey that can return a partial result if it does + * :: Experimental :: + * Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. */ + @Experimental def countByKeyApprox(timeout: Long, confidence: Double = 0.95) : PartialResult[Map[K, BoundedDouble]] = { self.map(_._1).countByValueApprox(timeout, confidence) } /** + * :: Experimental :: + * * Return approximate number of distinct values for each key in this RDD. - * The accuracy of approximation can be controlled through the relative standard deviation - * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vise versa. Uses the provided - * Partitioner to partition the output RDD. + * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * The relative accuracy is approximately `1.054 / sqrt(2^p)`. Setting a nonzero `sp > p` + * would trigger sparse representation of registers, which may reduce the memory consumption + * and increase accuracy when the cardinality is small. + * + * @param p The precision value for the normal set. + * `p` must be a value between 4 and `sp` if `sp` is not zero (32 max). + * @param sp The precision value for the sparse set, between 0 and 32. + * If `sp` equals 0, the sparse representation is skipped. + * @param partitioner Partitioner to use for the resulting RDD. + */ + @Experimental + def countApproxDistinctByKey(p: Int, sp: Int, partitioner: Partitioner): RDD[(K, Long)] = { + 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)") + val createHLL = (v: V) => { + val hll = new HyperLogLogPlus(p, sp) + hll.offer(v) + hll + } + val mergeValueHLL = (hll: HyperLogLogPlus, v: V) => { + hll.offer(v) + hll + } + val mergeHLL = (h1: HyperLogLogPlus, h2: HyperLogLogPlus) => { + h1.addAll(h2) + h1 + } + + combineByKey(createHLL, mergeValueHLL, mergeHLL, partitioner).mapValues(_.cardinality()) + } + + /** + * Return approximate number of distinct values for each key in this RDD. + * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * It must be greater than 0.000017. + * @param partitioner partitioner of the resulting RDD */ def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): RDD[(K, Long)] = { - val createHLL = (v: V) => new SerializableHyperLogLog(new HyperLogLog(relativeSD)).add(v) - val mergeValueHLL = (hll: SerializableHyperLogLog, v: V) => hll.add(v) - val mergeHLL = (h1: SerializableHyperLogLog, h2: SerializableHyperLogLog) => h1.merge(h2) - - combineByKey(createHLL, mergeValueHLL, mergeHLL, partitioner).mapValues(_.value.cardinality()) + 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) + countApproxDistinctByKey(if (p < 4) 4 else p, 0, partitioner) } /** * Return approximate number of distinct values for each key in this RDD. - * The accuracy of approximation can be controlled through the relative standard deviation - * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vise versa. HashPartitions the - * output RDD into numPartitions. * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @param relativeSD Relative accuracy. Smaller values create counters that require more space. + * 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)] = { countApproxDistinctByKey(relativeSD, new HashPartitioner(numPartitions)) } /** - * Return approximate number of distinct values for each key this RDD. - * The accuracy of approximation can be controlled through the relative standard deviation - * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vise versa. The default value of - * relativeSD is 0.05. Hash-partitions the output RDD using the existing partitioner/parallelism - * level. + * Return approximate number of distinct values for each key in this RDD. + * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @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)] = { countApproxDistinctByKey(relativeSD, defaultPartitioner(self)) @@ -260,8 +362,12 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) /** * Group the values for each key in the RDD into a single sequence. Allows controlling the * partitioning of the resulting key-value pair RDD by passing a Partitioner. + * + * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over + * each key, using [[PairRDDFunctions.reduceByKey]] or [[PairRDDFunctions.combineByKey]] + * will provide much better performance. */ - def groupByKey(partitioner: Partitioner): RDD[(K, Seq[V])] = { + def groupByKey(partitioner: Partitioner): RDD[(K, Iterable[V])] = { // 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. @@ -270,14 +376,18 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def mergeCombiners(c1: ArrayBuffer[V], c2: ArrayBuffer[V]) = c1 ++ c2 val bufs = combineByKey[ArrayBuffer[V]]( createCombiner _, mergeValue _, mergeCombiners _, partitioner, mapSideCombine=false) - bufs.asInstanceOf[RDD[(K, Seq[V])]] + bufs.mapValues(_.toIterable) } /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with into `numPartitions` partitions. + * + * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over + * each key, using [[PairRDDFunctions.reduceByKey]] or [[PairRDDFunctions.combineByKey]] + * will provide much better performance. */ - def groupByKey(numPartitions: Int): RDD[(K, Seq[V])] = { + def groupByKey(numPartitions: Int): RDD[(K, Iterable[V])] = { groupByKey(new HashPartitioner(numPartitions)) } @@ -285,10 +395,14 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * Return a copy of the RDD partitioned using the specified partitioner. */ def partitionBy(partitioner: Partitioner): RDD[(K, V)] = { - if (getKeyClass().isArray && partitioner.isInstanceOf[HashPartitioner]) { + if (keyClass.isArray && partitioner.isInstanceOf[HashPartitioner]) { throw new SparkException("Default partitioner cannot partition array keys.") } - if (self.partitioner == partitioner) self else new ShuffledRDD[K, V, (K, V)](self, partitioner) + if (self.partitioner == Some(partitioner)) { + self + } else { + new ShuffledRDD[K, V, (K, V)](self, partitioner) + } } /** @@ -298,7 +412,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => - for (v <- vs.iterator; w <- ws.iterator) yield (v, w) + for (v <- vs; w <- ws) yield (v, w) } } @@ -311,9 +425,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => if (ws.isEmpty) { - vs.iterator.map(v => (v, None)) + vs.map(v => (v, None)) } else { - for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w)) + for (v <- vs; w <- ws) yield (v, Some(w)) } } } @@ -328,9 +442,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) : RDD[(K, (Option[V], W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => if (vs.isEmpty) { - ws.iterator.map(w => (None, w)) + ws.map(w => (None, w)) } else { - for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w) + for (v <- vs; w <- ws) yield (Some(v), w) } } } @@ -357,8 +471,12 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with the existing partitioner/parallelism level. + * + * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over + * each key, using [[PairRDDFunctions.reduceByKey]] or [[PairRDDFunctions.combineByKey]] + * will provide much better performance, */ - def groupByKey(): RDD[(K, Seq[V])] = { + def groupByKey(): RDD[(K, Iterable[V])] = { groupByKey(defaultPartitioner(self)) } @@ -449,12 +567,35 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) new FlatMappedValuesRDD(self, cleanF) } + /** + * For each key k in `this` or `other1` or `other2` or `other3`, + * return a resulting RDD that contains a tuple with the list of values + * 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)], + partitioner: Partitioner) + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))] = { + if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) { + throw new SparkException("Default partitioner cannot partition array keys.") + } + val cg = new CoGroupedRDD[K](Seq(self, other1, other2, other3), partitioner) + cg.mapValues { case Seq(vs, w1s, w2s, w3s) => + (vs.asInstanceOf[Seq[V]], + w1s.asInstanceOf[Seq[W1]], + w2s.asInstanceOf[Seq[W2]], + w3s.asInstanceOf[Seq[W3]]) + } + } + /** * 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)], partitioner: Partitioner): RDD[(K, (Seq[V], Seq[W]))] = { - if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { + def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner) + : RDD[(K, (Iterable[V], Iterable[W]))] = { + if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other), partitioner) @@ -468,21 +609,33 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](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, (Seq[V], Seq[W1], Seq[W2]))] = { - if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { + if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner) cg.mapValues { case Seq(vs, w1s, w2s) => - (vs.asInstanceOf[Seq[V]], w1s.asInstanceOf[Seq[W1]], w2s.asInstanceOf[Seq[W2]]) + (vs.asInstanceOf[Seq[V]], + w1s.asInstanceOf[Seq[W1]], + w2s.asInstanceOf[Seq[W2]]) } } + /** + * For each key k in `this` or `other1` or `other2` or `other3`, + * return a resulting RDD that contains a tuple with the list of values + * 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]))] = { + cogroup(other1, other2, other3, defaultPartitioner(self, other1, other2, other3)) + } + /** * 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, (Seq[V], Seq[W]))] = { + def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, defaultPartitioner(self, other)) } @@ -491,7 +644,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](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, (Seq[V], Seq[W1], Seq[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } @@ -499,7 +652,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](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, (Seq[V], Seq[W]))] = { + def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, new HashPartitioner(numPartitions)) } @@ -508,21 +661,40 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](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, (Seq[V], Seq[W1], Seq[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { cogroup(other1, other2, new HashPartitioner(numPartitions)) } + /** + * For each key k in `this` or `other1` or `other2` or `other3`, + * return a resulting RDD that contains a tuple with the list of values + * 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)], + numPartitions: Int) + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))] = { + cogroup(other1, other2, other3, new HashPartitioner(numPartitions)) + } + /** Alias for cogroup. */ - def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, defaultPartitioner(self, other)) } /** Alias for cogroup. */ def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { 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]))] = { + cogroup(other1, other2, other3, defaultPartitioner(self, other1, other2, other3)) + } + /** * Return an RDD with the pairs from `this` whose keys are not in `other`. * @@ -567,7 +739,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * supporting the key and value types K and V in this RDD. */ def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) { - saveAsHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]]) + saveAsHadoopFile(path, keyClass, valueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } /** @@ -578,7 +750,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def saveAsHadoopFile[F <: OutputFormat[K, V]]( path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassTag[F]) { val runtimeClass = fm.runtimeClass - saveAsHadoopFile(path, getKeyClass, getValueClass, runtimeClass.asInstanceOf[Class[F]], codec) + saveAsHadoopFile(path, keyClass, valueClass, runtimeClass.asInstanceOf[Class[F]], codec) } /** @@ -586,7 +758,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * (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]) { - saveAsNewAPIHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]]) + saveAsNewAPIHadoopFile(path, keyClass, valueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } /** @@ -666,7 +838,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) val outfmt = job.getOutputFormatClass val jobFormat = outfmt.newInstance - if (jobFormat.isInstanceOf[NewFileOutputFormat[_, _]]) { + if (self.conf.getBoolean("spark.hadoop.validateOutputSpecs", true)) { // FileOutputFormat ignores the filesystem parameter jobFormat.checkOutputSpecs(job) } @@ -687,11 +859,15 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) val committer = format.getOutputCommitter(hadoopContext) committer.setupTask(hadoopContext) val writer = format.getRecordWriter(hadoopContext).asInstanceOf[NewRecordWriter[K,V]] - while (iter.hasNext) { - val (k, v) = iter.next() - writer.write(k, v) + try { + while (iter.hasNext) { + val (k, v) = iter.next() + writer.write(k, v) + } + } + finally { + writer.close(hadoopContext) } - writer.close(hadoopContext) committer.commitTask(hadoopContext) return 1 } @@ -728,7 +904,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) logDebug("Saving as hadoop file of type (" + keyClass.getSimpleName + ", " + valueClass.getSimpleName + ")") - if (outputFormatInstance.isInstanceOf[FileOutputFormat[_, _]]) { + if (self.conf.getBoolean("spark.hadoop.validateOutputSpecs", true)) { // FileOutputFormat ignores the filesystem parameter val ignoredFs = FileSystem.get(conf) conf.getOutputFormat.checkOutputSpecs(ignoredFs, conf) @@ -744,15 +920,17 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) writer.setup(context.stageId, context.partitionId, attemptNumber) writer.open() - - var count = 0 - while(iter.hasNext) { - val record = iter.next() - count += 1 - writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef]) + try { + var count = 0 + while(iter.hasNext) { + val record = iter.next() + count += 1 + writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef]) + } + } + finally { + writer.close() } - - writer.close() writer.commit() } @@ -770,7 +948,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def values: RDD[V] = self.map(_._2) - private[spark] def getKeyClass() = implicitly[ClassTag[K]].runtimeClass + private[spark] def keyClass: Class[_] = kt.runtimeClass + + private[spark] def valueClass: Class[_] = vt.runtimeClass - private[spark] def getValueClass() = implicitly[ClassTag[V]].runtimeClass + private[spark] def keyOrdering: Option[Ordering[K]] = Option(ord) } 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 5f03d7d650a30..66c71bf7e8bb5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala @@ -77,7 +77,7 @@ private[spark] class ParallelCollectionPartition[T: ClassTag]( slice = in.readInt() val ser = sfactory.newInstance() - Utils.deserializeViaNestedStream(in, ser)(ds => values = ds.readObject()) + Utils.deserializeViaNestedStream(in, ser)(ds => values = ds.readObject[Seq[T]]()) } } } @@ -117,6 +117,15 @@ private object ParallelCollectionRDD { if (numSlices < 1) { throw new IllegalArgumentException("Positive number of slices required") } + // Sequences need to be sliced at the same set of index positions for operations + // like RDD.zip() to behave as expected + def positions(length: Long, numSlices: Int): Iterator[(Int, Int)] = { + (0 until numSlices).iterator.map(i => { + val start = ((i * length) / numSlices).toInt + val end = (((i + 1) * length) / numSlices).toInt + (start, end) + }) + } seq match { case r: Range.Inclusive => { val sign = if (r.step < 0) { @@ -128,18 +137,17 @@ private object ParallelCollectionRDD { r.start, r.end + sign, r.step).asInstanceOf[Seq[T]], numSlices) } case r: Range => { - (0 until numSlices).map(i => { - val start = ((i * r.length.toLong) / numSlices).toInt - val end = (((i + 1) * r.length.toLong) / numSlices).toInt - new Range(r.start + start * r.step, r.start + end * r.step, r.step) - }).asInstanceOf[Seq[Seq[T]]] + positions(r.length, numSlices).map({ + case (start, end) => + new Range(r.start + start * r.step, r.start + end * r.step, r.step) + }).toSeq.asInstanceOf[Seq[Seq[T]]] } case nr: NumericRange[_] => { // For ranges of Long, Double, BigInteger, etc val slices = new ArrayBuffer[Seq[T]](numSlices) - val sliceSize = (nr.size + numSlices - 1) / numSlices // Round up to catch everything var r = nr - for (i <- 0 until numSlices) { + for ((start, end) <- positions(nr.length, numSlices)) { + val sliceSize = end - start slices += r.take(sliceSize).asInstanceOf[Seq[T]] r = r.drop(sliceSize) } @@ -147,11 +155,10 @@ private object ParallelCollectionRDD { } case _ => { val array = seq.toArray // To prevent O(n^2) operations for List etc - (0 until numSlices).map(i => { - val start = ((i * array.length.toLong) / numSlices).toInt - val end = (((i + 1) * array.length.toLong) / numSlices).toInt - array.slice(start, end).toSeq - }) + positions(array.length, numSlices).map({ + case (start, end) => + array.slice(start, end).toSeq + }).toSeq } } } 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 b0440ca7f32cf..f781a8d776f2a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -20,8 +20,10 @@ package org.apache.spark.rdd import scala.reflect.ClassTag import org.apache.spark.{NarrowDependency, Partition, TaskContext} +import org.apache.spark.annotation.DeveloperApi -class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends Partition { +private[spark] class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) + extends Partition { override val index = idx } @@ -30,7 +32,7 @@ class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends * Represents a dependency between the PartitionPruningRDD and its parent. In this * case, the child RDD contains a subset of partitions of the parents'. */ -class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boolean) +private[spark] class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boolean) extends NarrowDependency[T](rdd) { @transient @@ -45,11 +47,13 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo /** + * :: DeveloperApi :: * A RDD used to prune RDD partitions/partitions so we can avoid launching tasks on * all partitions. An example use case: If we know the RDD is partitioned by range, * and the execution DAG has a filter on the key, we can avoid launching tasks * on partitions that don't have the range covering the key. */ +@DeveloperApi class PartitionPruningRDD[T: ClassTag]( @transient prev: RDD[T], @transient partitionFilterFunc: Int => Boolean) @@ -63,6 +67,7 @@ class PartitionPruningRDD[T: ClassTag]( } +@DeveloperApi object PartitionPruningRDD { /** 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 a84357b38414e..0c2cd7a24783b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala @@ -33,7 +33,7 @@ class PartitionerAwareUnionRDDPartition( val idx: Int ) extends Partition { var parents = rdds.map(_.partitions(idx)).toArray - + override val index = idx override def hashCode(): Int = idx diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala index ce4c0d382baab..b5b8a5706deb3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala @@ -23,6 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.{Partition, TaskContext} import org.apache.spark.util.random.RandomSampler +import org.apache.spark.util.Utils private[spark] class PartitionwiseSampledRDDPartition(val prev: Partition, val seed: Long) @@ -38,14 +39,14 @@ class PartitionwiseSampledRDDPartition(val prev: Partition, val seed: Long) * * @param prev RDD to be sampled * @param sampler a random sampler - * @param seed random seed, default to System.nanoTime + * @param seed random seed * @tparam T input RDD item type * @tparam U sampled RDD item type */ -class PartitionwiseSampledRDD[T: ClassTag, U: ClassTag]( +private[spark] class PartitionwiseSampledRDD[T: ClassTag, U: ClassTag]( prev: RDD[T], sampler: RandomSampler[T, U], - @transient seed: Long = System.nanoTime) + @transient seed: Long = Utils.random.nextLong) extends RDD[U](prev) { override def getPartitions: Array[Partition] = { 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 41ae0fec823e7..5d77d37378458 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -37,7 +37,7 @@ import org.apache.spark.util.Utils * An RDD that pipes the contents of each parent partition through an external command * (printing them one per line) and returns the output as a collection of strings. */ -class PipedRDD[T: ClassTag]( +private[spark] class PipedRDD[T: ClassTag]( prev: RDD[T], command: Seq[String], envVars: Map[String, String], @@ -87,10 +87,10 @@ class PipedRDD[T: ClassTag]( // When spark.worker.separated.working.directory option is turned on, each // task will be run in separate directory. This should be resolve file // access conflict issue - val taskDirectory = "./tasks/" + java.util.UUID.randomUUID.toString + val taskDirectory = "tasks" + File.separator + java.util.UUID.randomUUID.toString var workInTaskDirectory = false logDebug("taskDirectory = " + taskDirectory) - if (separateWorkingDir == true) { + if (separateWorkingDir) { val currentDir = new File(".") logDebug("currentDir = " + currentDir.getAbsolutePath()) val taskDirFile = new File(taskDirectory) @@ -106,13 +106,13 @@ class PipedRDD[T: ClassTag]( for (file <- currentDir.list(tasksDirFilter)) { val fileWithDir = new File(currentDir, file) Utils.symlink(new File(fileWithDir.getAbsolutePath()), - new File(taskDirectory + "/" + fileWithDir.getName())) + new File(taskDirectory + File.separator + fileWithDir.getName())) } pb.directory(taskDirFile) workInTaskDirectory = true } catch { case e: Exception => logError("Unable to setup task working directory: " + e.getMessage + - " (" + taskDirectory + ")") + " (" + taskDirectory + ")", e) } } 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 08c42c5ee87b6..cebfd109d825f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -19,13 +19,11 @@ package org.apache.spark.rdd import java.util.Random -import scala.collection.Map -import scala.collection.JavaConversions.mapAsScalaMap +import scala.collection.{mutable, Map} import scala.collection.mutable.ArrayBuffer import scala.reflect.{classTag, ClassTag} -import com.clearspring.analytics.stream.cardinality.HyperLogLog -import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} +import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus import org.apache.hadoop.io.BytesWritable import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.io.NullWritable @@ -35,14 +33,16 @@ import org.apache.hadoop.mapred.TextOutputFormat import org.apache.spark._ import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext._ +import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator import org.apache.spark.partial.GroupedCountEvaluator import org.apache.spark.partial.PartialResult import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{BoundedPriorityQueue, SerializableHyperLogLog, Utils} -import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler} +import org.apache.spark.util.{BoundedPriorityQueue, CallSite, Utils} +import org.apache.spark.util.collection.OpenHashMap +import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, SamplingUtils} /** * A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. Represents an immutable, @@ -86,7 +86,11 @@ abstract class RDD[T: ClassTag]( // Methods that should be implemented by subclasses of RDD // ======================================================================= - /** Implemented by subclasses to compute a given partition. */ + /** + * :: DeveloperApi :: + * Implemented by subclasses to compute a given partition. + */ + @DeveloperApi def compute(split: Partition, context: TaskContext): Iterator[T] /** @@ -101,7 +105,9 @@ abstract class RDD[T: ClassTag]( */ protected def getDependencies: Seq[Dependency[_]] = deps - /** Optionally overridden by subclasses to specify placement preferences. */ + /** + * Optionally overridden by subclasses to specify placement preferences. + */ protected def getPreferredLocations(split: Partition): Seq[String] = Nil /** Optionally overridden by subclasses to specify how they are partitioned. */ @@ -121,7 +127,7 @@ abstract class RDD[T: ClassTag]( @transient var name: String = null /** Assign a name to this RDD */ - def setName(_name: String): RDD[T] = { + def setName(_name: String): this.type = { name = _name this } @@ -131,22 +137,24 @@ abstract class RDD[T: ClassTag]( * 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.. */ - def persist(newLevel: StorageLevel): RDD[T] = { + def persist(newLevel: StorageLevel): this.type = { // TODO: Handle changes of StorageLevel if (storageLevel != StorageLevel.NONE && newLevel != storageLevel) { throw new UnsupportedOperationException( "Cannot change storage level of an RDD after it was already assigned a level") } sc.persistRDD(this) + // Register the RDD with the ContextCleaner for automatic GC-based cleanup + sc.cleaner.foreach(_.registerRDDForCleanup(this)) storageLevel = newLevel this } /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ - def persist(): RDD[T] = persist(StorageLevel.MEMORY_ONLY) + def persist(): this.type = persist(StorageLevel.MEMORY_ONLY) /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ - def cache(): RDD[T] = persist() + def cache(): this.type = persist() /** * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. @@ -154,9 +162,9 @@ abstract class RDD[T: ClassTag]( * @param blocking Whether to block until all blocks are deleted. * @return This RDD. */ - def unpersist(blocking: Boolean = true): RDD[T] = { + def unpersist(blocking: Boolean = true): this.type = { logInfo("Removing RDD " + id + " from persistence list") - sc.unpersistRDD(this, blocking) + sc.unpersistRDD(id, blocking) storageLevel = StorageLevel.NONE this } @@ -221,6 +229,30 @@ abstract class RDD[T: ClassTag]( } } + /** + * Return the ancestors of the given RDD that are related to it only through a sequence of + * narrow dependencies. This traverses the given RDD's dependency tree using DFS, but maintains + * no ordering on the RDDs returned. + */ + private[spark] def getNarrowAncestors: Seq[RDD[_]] = { + val ancestors = new mutable.HashSet[RDD[_]] + + def visit(rdd: RDD[_]) { + val narrowDependencies = rdd.dependencies.filter(_.isInstanceOf[NarrowDependency[_]]) + val narrowParents = narrowDependencies.map(_.rdd) + val narrowParentsNotVisited = narrowParents.filterNot(ancestors.contains) + narrowParentsNotVisited.foreach { parent => + ancestors.add(parent) + visit(parent) + } + } + + visit(this) + + // In case there is a cycle, do not include the root itself + ancestors.filterNot(_ == this).toSeq + } + /** * Compute an RDD partition or read it from a checkpoint if the RDD is checkpointing. */ @@ -251,7 +283,7 @@ abstract class RDD[T: ClassTag]( /** * Return a new RDD containing the distinct elements in this RDD. */ - def distinct(numPartitions: Int): RDD[T] = + def distinct(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T] = map(x => (x, null)).reduceByKey((x, y) => x, numPartitions).map(_._1) /** @@ -268,7 +300,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): RDD[T] = { + def repartition(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T] = { coalesce(numPartitions, shuffle = true) } @@ -292,13 +324,25 @@ abstract class RDD[T: ClassTag]( * coalesce(1000, shuffle = true) will result in 1000 partitions with the * data distributed using a hash partitioner. */ - def coalesce(numPartitions: Int, shuffle: Boolean = false): RDD[T] = { + def coalesce(numPartitions: Int, shuffle: Boolean = false)(implicit ord: Ordering[T] = null) + : RDD[T] = { if (shuffle) { + /** Distributes elements evenly across output partitions, starting from a random partition. */ + def distributePartition(index: Int, items: Iterator[T]): Iterator[(Int, T)] = { + var position = (new Random(index)).nextInt(numPartitions) + items.map { t => + // Note that the hash code of the key will just be the key itself. The HashPartitioner + // will mod it with the number of total partitions. + position = position + 1 + (position, t) + } + } + // include a shuffle step so that our upstream tasks are still distributed new CoalescedRDD( - new ShuffledRDD[T, Null, (T, Null)](map(x => (x, null)), + new ShuffledRDD[Int, T, (Int, T)](mapPartitionsWithIndex(distributePartition), new HashPartitioner(numPartitions)), - numPartitions).keys + numPartitions).values } else { new CoalescedRDD(this, numPartitions) } @@ -307,7 +351,9 @@ abstract class RDD[T: ClassTag]( /** * Return a sampled subset of this RDD. */ - def sample(withReplacement: Boolean, fraction: Double, seed: Int): RDD[T] = { + def sample(withReplacement: Boolean, + fraction: Double, + seed: Long = Utils.random.nextLong): RDD[T] = { require(fraction >= 0.0, "Invalid fraction value: " + fraction) if (withReplacement) { new PartitionwiseSampledRDD[T, T](this, new PoissonSampler[T](fraction), seed) @@ -320,11 +366,11 @@ abstract class RDD[T: ClassTag]( * Randomly splits this RDD with the provided weights. * * @param weights weights for splits, will be normalized if they don't sum to 1 - * @param seed random seed, default to System.nanoTime + * @param seed random seed * * @return split RDDs in an array */ - def randomSplit(weights: Array[Double], seed: Long = System.nanoTime): Array[RDD[T]] = { + def randomSplit(weights: Array[Double], seed: Long = Utils.random.nextLong): Array[RDD[T]] = { val sum = weights.sum val normalizedCumWeights = weights.map(_ / sum).scanLeft(0.0d)(_ + _) normalizedCumWeights.sliding(2).map { x => @@ -332,45 +378,56 @@ abstract class RDD[T: ClassTag]( }.toArray } - def takeSample(withReplacement: Boolean, num: Int, seed: Int): Array[T] = { - var fraction = 0.0 - var total = 0 - val multiplier = 3.0 - val initialCount = this.count() - var maxSelected = 0 + /** + * Return a fixed-size sampled subset of this RDD in an array + * + * @param withReplacement whether sampling is done with replacement + * @param num size of the returned sample + * @param seed seed for the random number generator + * @return sample of specified size in an array + */ + def takeSample(withReplacement: Boolean, + num: Int, + seed: Long = Utils.random.nextLong): Array[T] = { + val numStDev = 10.0 if (num < 0) { throw new IllegalArgumentException("Negative number of elements requested") + } else if (num == 0) { + return new Array[T](0) } + val initialCount = this.count() if (initialCount == 0) { return new Array[T](0) } - if (initialCount > Integer.MAX_VALUE - 1) { - maxSelected = Integer.MAX_VALUE - 1 - } else { - maxSelected = initialCount.toInt + val maxSampleSize = Int.MaxValue - (numStDev * math.sqrt(Int.MaxValue)).toInt + if (num > maxSampleSize) { + throw new IllegalArgumentException("Cannot support a sample size > Int.MaxValue - " + + s"$numStDev * math.sqrt(Int.MaxValue)") } - if (num > initialCount && !withReplacement) { - total = maxSelected - fraction = multiplier * (maxSelected + 1) / initialCount - } else { - fraction = multiplier * (num + 1) / initialCount - total = num + val rand = new Random(seed) + if (!withReplacement && num >= initialCount) { + return Utils.randomizeInPlace(this.collect(), rand) } - val rand = new Random(seed) + val fraction = SamplingUtils.computeFractionForSampleSize(num, initialCount, + withReplacement) + var samples = this.sample(withReplacement, fraction, rand.nextInt()).collect() // 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 the initial size - while (samples.length < total) { + var numIters = 0 + while (samples.length < num) { + logWarning(s"Needed to re-sample due to insufficient sample size. Repeat #$numIters") samples = this.sample(withReplacement, fraction, rand.nextInt()).collect() + numIters += 1 } - Utils.randomizeInPlace(samples, rand).take(total) + Utils.randomizeInPlace(samples, rand).take(num) } /** @@ -385,16 +442,29 @@ abstract class RDD[T: ClassTag]( */ def ++(other: RDD[T]): RDD[T] = this.union(other) + /** + * Return this RDD sorted by the given key function. + */ + def sortBy[K]( + f: (T) => K, + ascending: Boolean = true, + numPartitions: Int = this.partitions.size) + (implicit ord: Ordering[K], ctag: ClassTag[K]): RDD[T] = + this.keyBy[K](f) + .sortByKey(ascending, numPartitions) + .values + /** * Return the intersection of this RDD and another one. The output will not contain any duplicate * elements, even if the input RDDs did. * * Note that this method performs a shuffle internally. */ - def intersection(other: RDD[T]): RDD[T] = + def intersection(other: RDD[T]): RDD[T] = { this.map(v => (v, null)).cogroup(other.map(v => (v, null))) .filter { case (_, (leftGroup, rightGroup)) => leftGroup.nonEmpty && rightGroup.nonEmpty } .keys + } /** * Return the intersection of this RDD and another one. The output will not contain any duplicate @@ -404,10 +474,12 @@ abstract class RDD[T: ClassTag]( * * @param partitioner Partitioner to use for the resulting RDD */ - def intersection(other: RDD[T], partitioner: Partitioner): RDD[T] = + def intersection(other: RDD[T], partitioner: Partitioner)(implicit ord: Ordering[T] = null) + : RDD[T] = { this.map(v => (v, null)).cogroup(other.map(v => (v, null)), partitioner) .filter { case (_, (leftGroup, rightGroup)) => leftGroup.nonEmpty && rightGroup.nonEmpty } .keys + } /** * Return the intersection of this RDD and another one. The output will not contain any duplicate @@ -417,10 +489,11 @@ 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] = + 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 + } /** * Return an RDD created by coalescing all elements within each partition into an array. @@ -434,22 +507,25 @@ abstract class RDD[T: ClassTag]( def cartesian[U: ClassTag](other: RDD[U]): RDD[(T, U)] = new CartesianRDD(sc, this, other) /** - * Return an RDD of grouped items. + * Return an RDD of grouped items. Each group consists of a key and a sequence of elements + * mapping to that key. */ - def groupBy[K: ClassTag](f: T => K): RDD[(K, Seq[T])] = + def groupBy[K](f: T => K)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] = groupBy[K](f, defaultPartitioner(this)) /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K: ClassTag](f: T => K, numPartitions: Int): RDD[(K, Seq[T])] = + def groupBy[K](f: T => K, numPartitions: Int)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] = groupBy(f, new HashPartitioner(numPartitions)) /** - * Return an RDD of grouped items. + * Return an RDD of grouped items. Each group consists of a key and a sequence of elements + * mapping to that key. */ - def groupBy[K: ClassTag](f: T => K, p: Partitioner): RDD[(K, Seq[T])] = { + def groupBy[K](f: T => K, p: Partitioner)(implicit kt: ClassTag[K], ord: Ordering[K] = null) + : RDD[(K, Iterable[T])] = { val cleanF = sc.clean(f) this.map(t => (cleanF(t), t)).groupByKey(p) } @@ -516,9 +592,11 @@ abstract class RDD[T: ClassTag]( } /** + * :: DeveloperApi :: * Return a new RDD by applying a function to each partition of this RDD. This is a variant of * mapPartitions that also passes the TaskContext into the closure. */ + @DeveloperApi def mapPartitionsWithContext[U: ClassTag]( f: (TaskContext, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { @@ -598,7 +676,19 @@ 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)] = new ZippedRDD(sc, this, other) + def zip[U: ClassTag](other: RDD[U]): RDD[(T, U)] = { + zipPartitions(other, true) { (thisIter, otherIter) => + new Iterator[(T, U)] { + def hasNext = (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) + } + } + } /** * Zip this RDD's partitions with one (or more) RDD(s) and return a new RDD by @@ -661,6 +751,18 @@ abstract class RDD[T: ClassTag]( Array.concat(results: _*) } + /** + * Return an iterator that contains all of the elements in this RDD. + * + * The iterator will consume as much memory as the largest partition in this RDD. + */ + def toLocalIterator: Iterator[T] = { + def collectPartition(p: Int): Array[T] = { + sc.runJob(this, (iter: Iterator[T]) => iter.toArray, Seq(p), allowLocal = false).head + } + (0 until partitions.length).iterator.flatMap(i => collectPartition(i)) + } + /** * Return an array that contains all of the elements in this RDD. */ @@ -692,7 +794,7 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD with the elements from `this` that are not in `other`. */ - def subtract(other: RDD[T], p: Partitioner): RDD[T] = { + def subtract(other: RDD[T], p: Partitioner)(implicit ord: Ordering[T] = null): RDD[T] = { 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 @@ -778,9 +880,11 @@ abstract class RDD[T: ClassTag]( def count(): Long = sc.runJob(this, Utils.getIteratorSize _).sum /** - * (Experimental) Approximate version of count() that returns a potentially incomplete result + * :: Experimental :: + * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. */ + @Experimental def countApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { val countElements: (TaskContext, Iterator[T]) => Long = { (ctx, iter) => var result = 0L @@ -798,46 +902,47 @@ abstract class RDD[T: ClassTag]( * 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(): Map[T, Long] = { + def countByValue()(implicit ord: Ordering[T] = null): Map[T, Long] = { if (elementClassTag.runtimeClass.isArray) { throw new SparkException("countByValue() does not support arrays") } // TODO: This should perhaps be distributed by default. - def countPartition(iter: Iterator[T]): Iterator[OLMap[T]] = { - val map = new OLMap[T] - while (iter.hasNext) { - val v = iter.next() - map.put(v, map.getLong(v) + 1L) + def countPartition(iter: Iterator[T]): Iterator[OpenHashMap[T,Long]] = { + val map = new OpenHashMap[T,Long] + iter.foreach { + t => map.changeValue(t, 1L, _ + 1L) } Iterator(map) } - def mergeMaps(m1: OLMap[T], m2: OLMap[T]): OLMap[T] = { - val iter = m2.object2LongEntrySet.fastIterator() - while (iter.hasNext) { - val entry = iter.next() - m1.put(entry.getKey, m1.getLong(entry.getKey) + entry.getLongValue) + def mergeMaps(m1: OpenHashMap[T,Long], m2: OpenHashMap[T,Long]): OpenHashMap[T,Long] = { + m2.foreach { case (key, value) => + m1.changeValue(key, value, _ + value) } m1 } val myResult = mapPartitions(countPartition).reduce(mergeMaps) - myResult.asInstanceOf[java.util.Map[T, Long]] // Will be wrapped as a Scala mutable Map + // Convert to a Scala mutable map + val mutableResult = scala.collection.mutable.Map[T,Long]() + myResult.foreach { case (k, v) => mutableResult.put(k, v) } + mutableResult } /** - * (Experimental) Approximate version of countByValue(). + * :: Experimental :: + * Approximate version of countByValue(). */ - def countByValueApprox( - timeout: Long, - confidence: Double = 0.95 - ): PartialResult[Map[T, BoundedDouble]] = { + @Experimental + def countByValueApprox(timeout: Long, confidence: Double = 0.95) + (implicit ord: Ordering[T] = null) + : PartialResult[Map[T, BoundedDouble]] = + { if (elementClassTag.runtimeClass.isArray) { throw new SparkException("countByValueApprox() does not support arrays") } - val countPartition: (TaskContext, Iterator[T]) => OLMap[T] = { (ctx, iter) => - val map = new OLMap[T] - while (iter.hasNext) { - val v = iter.next() - map.put(v, map.getLong(v) + 1L) + val countPartition: (TaskContext, Iterator[T]) => OpenHashMap[T,Long] = { (ctx, iter) => + val map = new OpenHashMap[T,Long] + iter.foreach { + t => map.changeValue(t, 1L, _ + 1L) } map } @@ -845,17 +950,53 @@ abstract class RDD[T: ClassTag]( sc.runApproximateJob(this, countPartition, evaluator, timeout) } + /** + * :: Experimental :: + * Return approximate number of distinct elements in the RDD. + * + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * The relative accuracy is approximately `1.054 / sqrt(2^p)`. Setting a nonzero `sp > p` + * would trigger sparse representation of registers, which may reduce the memory consumption + * and increase accuracy when the cardinality is small. + * + * @param p The precision value for the normal set. + * `p` must be a value between 4 and `sp` if `sp` is not zero (32 max). + * @param sp The precision value for the sparse set, between 0 and 32. + * If `sp` equals 0, the sparse representation is skipped. + */ + @Experimental + def countApproxDistinct(p: Int, sp: Int): Long = { + require(p >= 4, s"p ($p) must be greater than 0") + 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)") + val zeroCounter = new HyperLogLogPlus(p, sp) + aggregate(zeroCounter)( + (hll: HyperLogLogPlus, v: T) => { + hll.offer(v) + hll + }, + (h1: HyperLogLogPlus, h2: HyperLogLogPlus) => { + h1.addAll(h2) + h2 + }).cardinality() + } + /** * Return approximate number of distinct elements in the RDD. * - * The accuracy of approximation can be controlled through the relative standard deviation - * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vise versa. The default value of - * relativeSD is 0.05. + * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: + * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available + * here. + * + * @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 = { - val zeroCounter = new SerializableHyperLogLog(new HyperLogLog(relativeSD)) - aggregate(zeroCounter)(_.add(_), _.merge(_)).value.cardinality() + val p = math.ceil(2.0 * math.log(1.054 / relativeSD) / math.log(2)).toInt + countApproxDistinct(p, 0) } /** @@ -933,11 +1074,11 @@ abstract class RDD[T: ClassTag]( * Returns the top K (largest) elements from this RDD as defined by the specified * implicit Ordering[T]. This does the opposite of [[takeOrdered]]. For example: * {{{ - * sc.parallelize([10, 4, 2, 12, 3]).top(1) - * // returns [12] + * sc.parallelize(Seq(10, 4, 2, 12, 3)).top(1) + * // returns Array(12) * - * sc.parallelize([2, 3, 4, 5, 6]).top(2) - * // returns [6, 5] + * sc.parallelize(Seq(2, 3, 4, 5, 6)).top(2) + * // returns Array(6, 5) * }}} * * @param num the number of top elements to return @@ -951,11 +1092,11 @@ abstract class RDD[T: ClassTag]( * implicit Ordering[T] and maintains the ordering. This does the opposite of [[top]]. * For example: * {{{ - * sc.parallelize([10, 4, 2, 12, 3]).takeOrdered(1) - * // returns [12] + * sc.parallelize(Seq(10, 4, 2, 12, 3)).takeOrdered(1) + * // returns Array(2) * - * sc.parallelize([2, 3, 4, 5, 6]).takeOrdered(2) - * // returns [2, 3] + * sc.parallelize(Seq(2, 3, 4, 5, 6)).takeOrdered(2) + * // returns Array(2, 3) * }}} * * @param num the number of top elements to return @@ -978,13 +1119,13 @@ 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 = 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 = this.reduce(ord.min) /** * Save this RDD as a text file, using string representations of elements. @@ -1060,8 +1201,8 @@ abstract class RDD[T: ClassTag]( private var storageLevel: StorageLevel = StorageLevel.NONE /** User code that created this RDD (e.g. `textFile`, `parallelize`). */ - @transient private[spark] val creationSiteInfo = Utils.getCallSiteInfo - private[spark] def getCreationSite: String = creationSiteInfo.toString + @transient private[spark] val creationSite = Utils.getCallSite + private[spark] def getCreationSite: String = Option(creationSite).map(_.short).getOrElse("") private[spark] def elementClassTag: ClassTag[T] = classTag[T] @@ -1079,9 +1220,9 @@ abstract class RDD[T: ClassTag]( @transient private var doCheckpointCalled = false /** - * Performs the checkpointing of this RDD by saving this. It is called by the DAGScheduler - * after a job using this RDD has completed (therefore the RDD has been materialized and - * potentially stored in memory). doCheckpoint() is called recursively on the parent RDDs. + * Performs the checkpointing of this RDD by saving this. It is called after a job using this RDD + * has completed (therefore the RDD has been materialized and potentially stored in memory). + * doCheckpoint() is called recursively on the parent RDDs. */ private[spark] def doCheckpoint() { if (!doCheckpointCalled) { @@ -1129,5 +1270,4 @@ abstract class RDD[T: ClassTag]( def toJavaRDD() : JavaRDD[T] = { new JavaRDD(this)(elementClassTag) } - } 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 953f0555e57c5..c3b2a33fb54d0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -92,7 +92,7 @@ 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(path.toString, broadcastedConf) _) + 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) { throw new SparkException( diff --git a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala index 4ceea557f569c..b097c30f8c231 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala @@ -33,7 +33,7 @@ class SampledRDDPartition(val prev: Partition, val seed: Int) extends Partition } @deprecated("Replaced by PartitionwiseSampledRDD", "1.0.0") -class SampledRDD[T: ClassTag]( +private[spark] class SampledRDD[T: ClassTag]( prev: RDD[T], withReplacement: Boolean, frac: Double, 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 7df9a2960d8a5..9a1efc83cbe6a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala @@ -68,8 +68,8 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag val keyClass = getWritableClass[K] val valueClass = getWritableClass[V] - val convertKey = !classOf[Writable].isAssignableFrom(self.getKeyClass) - val convertValue = !classOf[Writable].isAssignableFrom(self.getValueClass) + val convertKey = !classOf[Writable].isAssignableFrom(self.keyClass) + val convertValue = !classOf[Writable].isAssignableFrom(self.valueClass) logInfo("Saving as sequence file of type (" + keyClass.getSimpleName + "," + valueClass.getSimpleName + ")" ) 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 02660ea6a45c5..bb108ef163c56 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, Partitioner, ShuffleDependency, SparkEnv, TaskContext} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.serializer.Serializer private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { @@ -28,21 +29,24 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { } /** + * :: DeveloperApi :: * The resulting RDD from a shuffle (e.g. repartitioning of data). * @param prev the parent RDD. * @param part the partitioner used to partition the RDD * @tparam K the key class. * @tparam V the value class. */ +@DeveloperApi class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( @transient var prev: RDD[P], part: Partitioner) extends RDD[P](prev.context, Nil) { - private var serializer: Serializer = null + private var serializer: Option[Serializer] = None + /** Set a serializer for this RDD's shuffle, or null to use the default (spark.serializer) */ def setSerializer(serializer: Serializer): ShuffledRDD[K, V, P] = { - this.serializer = serializer + this.serializer = Option(serializer) this } @@ -57,9 +61,10 @@ class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( } override def compute(split: Partition, context: TaskContext): Iterator[P] = { - val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId - val ser = Serializer.getSerializer(serializer) - SparkEnv.get.shuffleFetcher.fetch[P](shuffledId, split.index, context, ser) + val dep = dependencies.head.asInstanceOf[ShuffleDependency[K, V, V]] + SparkEnv.get.shuffleManager.getReader(dep.shuffleHandle, split.index, split.index + 1, context) + .read() + .asInstanceOf[Iterator[P]] } override def clearDependencies() { 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 9a09c05bbc959..ed24ea22a661c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala @@ -54,10 +54,11 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( part: Partitioner) extends RDD[(K, V)](rdd1.context, Nil) { - private var serializer: Serializer = null + private var serializer: Option[Serializer] = None + /** Set a serializer for this RDD's shuffle, or null to use the default (spark.serializer) */ def setSerializer(serializer: Serializer): SubtractedRDD[K, V, W] = { - this.serializer = serializer + this.serializer = Option(serializer) this } @@ -79,8 +80,8 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( // 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.shuffleId) + case s: ShuffleDependency[_, _, _] => + new ShuffleCoGroupSplitDep(s.shuffleHandle) case _ => new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i)) } @@ -93,7 +94,6 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( override def compute(p: Partition, context: TaskContext): Iterator[(K, V)] = { val partition = p.asInstanceOf[CoGroupPartition] - val ser = Serializer.getSerializer(serializer) val map = new JHashMap[K, ArrayBuffer[V]] def getSeq(k: K): ArrayBuffer[V] = { val seq = map.get(k) @@ -109,9 +109,10 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( case NarrowCoGroupSplitDep(rdd, _, itsSplit) => rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, V]]].foreach(op) - case ShuffleCoGroupSplitDep(shuffleId) => - val iter = SparkEnv.get.shuffleFetcher.fetch[Product2[K, V]](shuffleId, partition.index, - context, ser) + case ShuffleCoGroupSplitDep(handle) => + val iter = SparkEnv.get.shuffleManager + .getReader(handle, partition.index, partition.index + 1, context) + .read() iter.foreach(op) } // the first dep is rdd1; add all values to the map 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 a447030752096..21c6e07d69f90 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, RangeDependency, SparkContext, TaskContext} +import org.apache.spark.annotation.DeveloperApi private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitIndex: Int) extends Partition { @@ -43,6 +44,7 @@ private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitInd } } +@DeveloperApi class UnionRDD[T: ClassTag]( sc: SparkContext, @transient var rdds: Seq[RDD[T]]) 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 b56643444aa40..f3d30f6c9b32f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -41,7 +41,7 @@ private[spark] class ZippedPartitionsPartition( } } -abstract class ZippedPartitionsBaseRDD[V: ClassTag]( +private[spark] abstract class ZippedPartitionsBaseRDD[V: ClassTag]( sc: SparkContext, var rdds: Seq[RDD[_]], preservesPartitioning: Boolean = false) @@ -74,7 +74,7 @@ abstract class ZippedPartitionsBaseRDD[V: ClassTag]( } } -class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag]( +private[spark] class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag]( sc: SparkContext, f: (Iterator[A], Iterator[B]) => Iterator[V], var rdd1: RDD[A], @@ -94,7 +94,7 @@ class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag]( } } -class ZippedPartitionsRDD3 +private[spark] class ZippedPartitionsRDD3 [A: ClassTag, B: ClassTag, C: ClassTag, V: ClassTag]( sc: SparkContext, f: (Iterator[A], Iterator[B], Iterator[C]) => Iterator[V], @@ -119,7 +119,7 @@ class ZippedPartitionsRDD3 } } -class ZippedPartitionsRDD4 +private[spark] class ZippedPartitionsRDD4 [A: ClassTag, B: ClassTag, C: ClassTag, D:ClassTag, V: ClassTag]( sc: SparkContext, f: (Iterator[A], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V], diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala deleted file mode 100644 index 2119e76f0e032..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala +++ /dev/null @@ -1,87 +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.rdd - -import java.io.{IOException, ObjectOutputStream} - -import scala.reflect.ClassTag - -import org.apache.spark.{OneToOneDependency, Partition, SparkContext, TaskContext} - -private[spark] class ZippedPartition[T: ClassTag, U: ClassTag]( - idx: Int, - @transient rdd1: RDD[T], - @transient rdd2: RDD[U] - ) extends Partition { - - var partition1 = rdd1.partitions(idx) - var partition2 = rdd2.partitions(idx) - override val index: Int = idx - - def partitions = (partition1, partition2) - - @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { - // Update the reference to parent partition at the time of task serialization - partition1 = rdd1.partitions(idx) - partition2 = rdd2.partitions(idx) - oos.defaultWriteObject() - } -} - -class ZippedRDD[T: ClassTag, U: ClassTag]( - sc: SparkContext, - var rdd1: RDD[T], - var rdd2: RDD[U]) - extends RDD[(T, U)](sc, List(new OneToOneDependency(rdd1), new OneToOneDependency(rdd2))) { - - override def getPartitions: Array[Partition] = { - if (rdd1.partitions.size != rdd2.partitions.size) { - throw new IllegalArgumentException("Can't zip RDDs with unequal numbers of partitions") - } - val array = new Array[Partition](rdd1.partitions.size) - for (i <- 0 until rdd1.partitions.size) { - array(i) = new ZippedPartition(i, rdd1, rdd2) - } - array - } - - override def compute(s: Partition, context: TaskContext): Iterator[(T, U)] = { - val (partition1, partition2) = s.asInstanceOf[ZippedPartition[T, U]].partitions - rdd1.iterator(partition1, context).zip(rdd2.iterator(partition2, context)) - } - - override def getPreferredLocations(s: Partition): Seq[String] = { - val (partition1, partition2) = s.asInstanceOf[ZippedPartition[T, U]].partitions - val pref1 = rdd1.preferredLocations(partition1) - val pref2 = rdd2.preferredLocations(partition2) - // Check whether there are any hosts that match both RDDs; otherwise return the union - val exactMatchLocations = pref1.intersect(pref2) - if (!exactMatchLocations.isEmpty) { - exactMatchLocations - } else { - (pref1 ++ pref2).distinct - } - } - - override def clearDependencies() { - super.clearDependencies() - rdd1 = null - rdd2 = null - } -} diff --git a/core/src/main/scala/org/apache/spark/rdd/package-info.java b/core/src/main/scala/org/apache/spark/rdd/package-info.java new file mode 100644 index 0000000000000..176cc58179fb0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/package-info.java @@ -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. + */ + +/** + * Provides implementation's of various RDDs. + */ +package org.apache.spark.rdd; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/rdd/package.scala b/core/src/main/scala/org/apache/spark/rdd/package.scala new file mode 100644 index 0000000000000..55fc6e4d2b4df --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Provides several RDD implementations. See [[org.apache.spark.rdd.RDD]]. + */ +package object rdd 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 9257f48559c9e..b755d8fb15757 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import java.util.Properties import org.apache.spark.TaskContext +import org.apache.spark.util.CallSite /** * Tracks information about an active job in the DAGScheduler. @@ -29,7 +30,7 @@ private[spark] class ActiveJob( val finalStage: Stage, val func: (TaskContext, Iterator[_]) => _, val partitions: Array[Int], - val callSite: String, + val callSite: CallSite, val listener: JobListener, val properties: Properties) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala new file mode 100644 index 0000000000000..cd5d44ad4a7e6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.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.scheduler + +/** + * A simple listener for application events. + * + * This listener expects to hear events from a single application only. If events + * from multiple applications are seen, the behavior is unspecified. + */ +private[spark] class ApplicationEventListener extends SparkListener { + var appName = "" + var sparkUser = "" + var startTime = -1L + var endTime = -1L + var viewAcls = "" + var enableViewAcls = false + + def applicationStarted = startTime != -1 + + def applicationCompleted = endTime != -1 + + def applicationDuration: Long = { + val difference = endTime - startTime + if (applicationStarted && applicationCompleted && difference > 0) difference else -1L + } + + override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { + appName = applicationStart.appName + startTime = applicationStart.time + sparkUser = applicationStart.sparkUser + } + + override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { + endTime = applicationEnd.time + } + + override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { + synchronized { + val environmentDetails = environmentUpdate.environmentDetails + val allProperties = environmentDetails("Spark Properties").toMap + viewAcls = allProperties.getOrElse("spark.ui.view.acls", "") + enableViewAcls = allProperties.getOrElse("spark.ui.acls.enable", "false").toBoolean + } + } +} 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 ef3d24d746829..b3ebaa547de0d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -17,22 +17,28 @@ package org.apache.spark.scheduler -import java.io.NotSerializableException +import java.io.{NotSerializableException, PrintWriter, StringWriter} import java.util.Properties import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} +import scala.concurrent.Await import scala.concurrent.duration._ +import scala.language.postfixOps import scala.reflect.ClassTag import akka.actor._ +import akka.actor.OneForOneStrategy +import akka.actor.SupervisorStrategy.Stop +import akka.pattern.ask +import akka.util.Timeout import org.apache.spark._ 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.{BlockId, BlockManager, BlockManagerMaster, RDDBlockId} -import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} +import org.apache.spark.util.{CallSite, SystemClock, Clock, Utils} /** * The high-level scheduling layer that implements stage-oriented scheduling. It computes a DAG of @@ -47,24 +53,23 @@ import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedH * 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. * - * THREADING: This class runs all its logic in a single thread executing the run() method, to which - * events are submitted using a synchronized queue (eventQueue). The public API methods, such as - * runJob, taskEnded and executorLost, post events asynchronously to this queue. All other methods - * should be private. */ private[spark] class DAGScheduler( - taskScheduler: TaskScheduler, + private[scheduler] val sc: SparkContext, + private[scheduler] val taskScheduler: TaskScheduler, listenerBus: LiveListenerBus, mapOutputTracker: MapOutputTrackerMaster, blockManagerMaster: BlockManagerMaster, - env: SparkEnv) + env: SparkEnv, + clock: Clock = SystemClock) extends Logging { import DAGScheduler._ def this(sc: SparkContext, taskScheduler: TaskScheduler) = { this( + sc, taskScheduler, sc.listenerBus, sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], @@ -74,19 +79,17 @@ class DAGScheduler( def this(sc: SparkContext) = this(sc, sc.taskScheduler) - private var eventProcessActor: ActorRef = _ - private[scheduler] val nextJobId = new AtomicInteger(0) private[scheduler] def numTotalJobs: Int = nextJobId.get() private val nextStageId = new AtomicInteger(0) - private[scheduler] val jobIdToStageIds = new TimeStampedHashMap[Int, HashSet[Int]] - private[scheduler] val stageIdToJobIds = new TimeStampedHashMap[Int, HashSet[Int]] - private[scheduler] val stageIdToStage = new TimeStampedHashMap[Int, Stage] - private[scheduler] val shuffleToMapStage = new TimeStampedHashMap[Int, Stage] + private[scheduler] val jobIdToStageIds = new HashMap[Int, HashSet[Int]] + private[scheduler] val stageIdToJobIds = new HashMap[Int, HashSet[Int]] + private[scheduler] val stageIdToStage = new HashMap[Int, Stage] + private[scheduler] val shuffleToMapStage = new HashMap[Int, Stage] private[scheduler] val jobIdToActiveJob = new HashMap[Int, ActiveJob] private[scheduler] val resultStageToJob = new HashMap[Stage, ActiveJob] - private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] + private[scheduler] val stageToInfos = new HashMap[Stage, StageInfo] // Stages we need to run whose parents aren't done private[scheduler] val waitingStages = new HashSet[Stage] @@ -98,7 +101,7 @@ class DAGScheduler( private[scheduler] val failedStages = new HashSet[Stage] // Missing tasks from each stage - private[scheduler] val pendingTasks = new TimeStampedHashMap[Stage, HashSet[Task[_]]] + private[scheduler] val pendingTasks = new HashMap[Stage, HashSet[Task[_]]] private[scheduler] val activeJobs = new HashSet[ActiveJob] @@ -113,53 +116,31 @@ class DAGScheduler( // stray messages to detect. private val failedEpoch = new HashMap[String, Long] - private val metadataCleaner = - new MetadataCleaner(MetadataCleanerType.DAG_SCHEDULER, this.cleanup, env.conf) + private val dagSchedulerActorSupervisor = + env.actorSystem.actorOf(Props(new DAGSchedulerActorSupervisor(this))) - taskScheduler.setDAGScheduler(this) + private[scheduler] var eventProcessActor: ActorRef = _ - /** - * Starts the event processing actor. The actor has two responsibilities: - * - * 1. Waits for events like job submission, task finished, task failure etc., and calls - * [[org.apache.spark.scheduler.DAGScheduler.processEvent()]] to process them. - * 2. Schedules a periodical task to resubmit failed stages. - * - * NOTE: the actor cannot be started in the constructor, because the periodical task references - * some internal states of the enclosing [[org.apache.spark.scheduler.DAGScheduler]] object, thus - * cannot be scheduled until the [[org.apache.spark.scheduler.DAGScheduler]] is fully constructed. - */ - def start() { - eventProcessActor = env.actorSystem.actorOf(Props(new Actor { - /** - * The main event loop of the DAG scheduler. - */ - def receive = { - case event: DAGSchedulerEvent => - logTrace("Got event of type " + event.getClass.getName) - - /** - * All events are forwarded to `processEvent()`, so that the event processing logic can - * easily tested without starting a dedicated actor. Please refer to `DAGSchedulerSuite` - * for details. - */ - if (!processEvent(event)) { - submitWaitingStages() - } else { - context.stop(self) - } - } - })) + private def initializeEventProcessActor() { + // blocking the thread until supervisor is started, which ensures eventProcessActor is + // not null before any job is submitted + implicit val timeout = Timeout(30 seconds) + val initEventActorReply = + dagSchedulerActorSupervisor ? Props(new DAGSchedulerEventProcessActor(this)) + eventProcessActor = Await.result(initEventActorReply, timeout.duration). + asInstanceOf[ActorRef] } + initializeEventProcessActor() + // Called by TaskScheduler to report task's starting. def taskStarted(task: Task[_], taskInfo: TaskInfo) { eventProcessActor ! BeginEvent(task, taskInfo) } // Called to report that a task has completed and results are being fetched remotely. - def taskGettingResult(task: Task[_], taskInfo: TaskInfo) { - eventProcessActor ! GettingResultEvent(task, taskInfo) + def taskGettingResult(taskInfo: TaskInfo) { + eventProcessActor ! GettingResultEvent(taskInfo) } // Called by TaskScheduler to report task completions or failures. @@ -209,12 +190,14 @@ 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): Stage = { shuffleToMapStage.get(shuffleDep.shuffleId) match { case Some(stage) => stage case None => val stage = - newOrUsedStage(shuffleDep.rdd, shuffleDep.rdd.partitions.size, shuffleDep, jobId) + newOrUsedStage( + shuffleDep.rdd, shuffleDep.rdd.partitions.size, shuffleDep, jobId, + shuffleDep.rdd.creationSite) shuffleToMapStage(shuffleDep.shuffleId) = stage stage } @@ -229,9 +212,9 @@ class DAGScheduler( private def newStage( rdd: RDD[_], numTasks: Int, - shuffleDep: Option[ShuffleDependency[_,_]], + shuffleDep: Option[ShuffleDependency[_, _, _]], jobId: Int, - callSite: Option[String] = None) + callSite: CallSite) : Stage = { val id = nextStageId.getAndIncrement() @@ -252,13 +235,13 @@ class DAGScheduler( private def newOrUsedStage( rdd: RDD[_], numTasks: Int, - shuffleDep: ShuffleDependency[_,_], + shuffleDep: ShuffleDependency[_, _, _], jobId: Int, - callSite: Option[String] = None) + callSite: CallSite) : Stage = { val stage = newStage(rdd, numTasks, Some(shuffleDep), jobId, callSite) - if (mapOutputTracker.has(shuffleDep.shuffleId)) { + if (mapOutputTracker.containsShuffle(shuffleDep.shuffleId)) { val serLocs = mapOutputTracker.getSerializedMapOutputStatuses(shuffleDep.shuffleId) val locs = MapOutputTracker.deserializeMapStatuses(serLocs) for (i <- 0 until locs.size) { @@ -288,7 +271,7 @@ class DAGScheduler( // we can't do it in its constructor because # of partitions is unknown for (dep <- r.dependencies) { dep match { - case shufDep: ShuffleDependency[_,_] => + case shufDep: ShuffleDependency[_, _, _] => parents += getShuffleMapStage(shufDep, jobId) case _ => visit(dep.rdd) @@ -309,7 +292,7 @@ class DAGScheduler( if (getCacheLocs(rdd).contains(Nil)) { for (dep <- rdd.dependencies) { dep match { - case shufDep: ShuffleDependency[_,_] => + case shufDep: ShuffleDependency[_, _, _] => val mapStage = getShuffleMapStage(shufDep, stage.jobId) if (!mapStage.isAvailable) { missing += mapStage @@ -345,22 +328,24 @@ class DAGScheduler( } /** - * Removes job and any stages that are not needed by any other job. Returns the set of ids for - * stages that were removed. The associated tasks for those stages need to be cancelled if we - * got here via job cancellation. + * Removes state for job and any stages that are not needed by any other job. Does not + * handle cancelling tasks or notifying the SparkListener about finished jobs/stages/tasks. + * + * @param job The job whose state to cleanup. + * @param resultStage Specifies the result stage for the job; if set to None, this method + * searches resultStagesToJob to find and cleanup the appropriate result stage. */ - private def removeJobAndIndependentStages(jobId: Int): Set[Int] = { - val registeredStages = jobIdToStageIds(jobId) - val independentStages = new HashSet[Int]() - if (registeredStages.isEmpty) { - logError("No stages registered for job " + jobId) + private def cleanupStateForJobAndIndependentStages(job: ActiveJob, resultStage: Option[Stage]) { + val registeredStages = jobIdToStageIds.get(job.jobId) + if (registeredStages.isEmpty || registeredStages.get.isEmpty) { + logError("No stages registered for job " + job.jobId) } else { - stageIdToJobIds.filterKeys(stageId => registeredStages.contains(stageId)).foreach { + stageIdToJobIds.filterKeys(stageId => registeredStages.get.contains(stageId)).foreach { case (stageId, jobSet) => - if (!jobSet.contains(jobId)) { + if (!jobSet.contains(job.jobId)) { logError( "Job %d not registered for stage %d even though that stage was registered for the job" - .format(jobId, stageId)) + .format(job.jobId, stageId)) } else { def removeStage(stageId: Int) { // data structures based on Stage @@ -390,27 +375,35 @@ class DAGScheduler( stageIdToStage -= stageId stageIdToJobIds -= stageId + ShuffleMapTask.removeStage(stageId) + ResultTask.removeStage(stageId) + logDebug("After removal of stage %d, remaining stages = %d" .format(stageId, stageIdToStage.size)) } - jobSet -= jobId + jobSet -= job.jobId if (jobSet.isEmpty) { // no other job needs this stage - independentStages += stageId removeStage(stageId) } } } } - independentStages.toSet - } - - private def jobIdToStageIdsRemove(jobId: Int) { - if (!jobIdToStageIds.contains(jobId)) { - logDebug("Trying to remove unregistered job " + jobId) + jobIdToStageIds -= job.jobId + jobIdToActiveJob -= job.jobId + activeJobs -= job + + if (resultStage.isEmpty) { + // Clean up result stages. + val resultStagesForJob = resultStageToJob.keySet.filter( + stage => resultStageToJob(stage).jobId == job.jobId) + if (resultStagesForJob.size != 1) { + logWarning( + s"${resultStagesForJob.size} result stages for job ${job.jobId} (expect exactly 1)") + } + resultStageToJob --= resultStagesForJob } else { - removeJobAndIndependentStages(jobId) - jobIdToStageIds -= jobId + resultStageToJob -= resultStage.get } } @@ -422,14 +415,14 @@ class DAGScheduler( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, partitions: Seq[Int], - callSite: String, + callSite: CallSite, allowLocal: Boolean, resultHandler: (Int, U) => Unit, properties: Properties = null): 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).foreach { p => + partitions.find(p => p >= maxPartitions || p < 0).foreach { p => throw new IllegalArgumentException( "Attempting to access a non-existent partition: " + p + ". " + "Total number of partitions: " + maxPartitions) @@ -452,7 +445,7 @@ class DAGScheduler( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, partitions: Seq[Int], - callSite: String, + callSite: CallSite, allowLocal: Boolean, resultHandler: (Int, U) => Unit, properties: Properties = null) @@ -460,8 +453,8 @@ class DAGScheduler( val waiter = submitJob(rdd, func, partitions, callSite, allowLocal, resultHandler, properties) waiter.awaitResult() match { case JobSucceeded => {} - case JobFailed(exception: Exception, _) => - logInfo("Failed to run " + callSite) + case JobFailed(exception: Exception) => + logInfo("Failed to run " + callSite.short) throw exception } } @@ -470,7 +463,7 @@ class DAGScheduler( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, evaluator: ApproximateEvaluator[U, R], - callSite: String, + callSite: CallSite, timeout: Long, properties: Properties = null) : PartialResult[R] = @@ -504,113 +497,20 @@ class DAGScheduler( eventProcessActor ! AllJobsCancelled } + private[scheduler] def doCancelAllJobs() { + // Cancel all running jobs. + runningStages.map(_.jobId).foreach(handleJobCancellation(_, + reason = "as part of cancellation of all jobs")) + activeJobs.clear() // These should already be empty by this point, + jobIdToActiveJob.clear() // but just in case we lost track of some jobs... + submitWaitingStages() + } + /** - * Process one event retrieved from the event processing actor. - * - * @param event The event to be processed. - * @return `true` if we should stop the event loop. + * Cancel all jobs associated with a running or scheduled stage. */ - private[scheduler] def processEvent(event: DAGSchedulerEvent): Boolean = { - event match { - case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) => - var finalStage: Stage = 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(rdd, partitions.size, None, jobId, Some(callSite)) - } catch { - case e: Exception => - logWarning("Creating new stage failed due to exception - job: " + jobId, e) - listener.jobFailed(e) - return false - } - val job = new ActiveJob(jobId, finalStage, func, partitions, callSite, listener, properties) - clearCacheLocs() - logInfo("Got job " + job.jobId + " (" + callSite + ") with " + partitions.length + - " output partitions (allowLocal=" + allowLocal + ")") - logInfo("Final stage: " + finalStage + " (" + finalStage.name + ")") - logInfo("Parents of final stage: " + finalStage.parents) - logInfo("Missing parents: " + getMissingParentStages(finalStage)) - if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { - // Compute very short actions like first() or take() with no parent stages locally. - listenerBus.post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) - runLocally(job) - } else { - jobIdToActiveJob(jobId) = job - activeJobs += job - resultStageToJob(finalStage) = job - listenerBus.post( - SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, properties)) - submitStage(finalStage) - } - - case JobCancelled(jobId) => - handleJobCancellation(jobId) - - case JobGroupCancelled(groupId) => - // 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 jobIds = activeInGroup.map(_.jobId) - jobIds.foreach(handleJobCancellation) - - case AllJobsCancelled => - // Cancel all running jobs. - runningStages.map(_.jobId).foreach(handleJobCancellation) - activeJobs.clear() // These should already be empty by this point, - jobIdToActiveJob.clear() // but just in case we lost track of some jobs... - - case ExecutorAdded(execId, host) => - handleExecutorAdded(execId, host) - - case ExecutorLost(execId) => - handleExecutorLost(execId) - - case BeginEvent(task, taskInfo) => - for ( - stage <- stageIdToStage.get(task.stageId); - stageInfo <- stageToInfos.get(stage) - ) { - if (taskInfo.serializedSize > TASK_SIZE_TO_WARN * 1024 && - !stageInfo.emittedTaskSizeWarning) { - stageInfo.emittedTaskSizeWarning = true - logWarning(("Stage %d (%s) contains a task of very large " + - "size (%d KB). The maximum recommended task size is %d KB.").format( - task.stageId, stageInfo.name, taskInfo.serializedSize / 1024, TASK_SIZE_TO_WARN)) - } - } - listenerBus.post(SparkListenerTaskStart(task.stageId, taskInfo)) - - case GettingResultEvent(task, taskInfo) => - listenerBus.post(SparkListenerTaskGettingResult(taskInfo)) - - case completion @ CompletionEvent(task, reason, _, _, taskInfo, taskMetrics) => - val stageId = task.stageId - val taskType = Utils.getFormattedClassName(task) - listenerBus.post(SparkListenerTaskEnd(stageId, taskType, reason, taskInfo, taskMetrics)) - handleTaskCompletion(completion) - - case TaskSetFailed(taskSet, reason) => - stageIdToStage.get(taskSet.stageId).foreach { abortStage(_, reason) } - - case ResubmitFailedStages => - if (failedStages.size > 0) { - // Failed stages may be removed by job cancellation, so failed might be empty even if - // the ResubmitFailedStages event has been scheduled. - resubmitFailedStages() - } - - case StopDAGScheduler => - // Cancel any active jobs - for (job <- activeJobs) { - val error = new SparkException("Job cancelled because SparkContext was shut down") - job.listener.jobFailed(error) - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, -1))) - } - return true - } - false + def cancelStage(stageId: Int) { + eventProcessActor ! StageCancelled(stageId) } /** @@ -618,20 +518,25 @@ class DAGScheduler( * the last fetch failure. */ private[scheduler] def resubmitFailedStages() { - logInfo("Resubmitting failed stages") - clearCacheLocs() - val failedStagesCopy = failedStages.toArray - failedStages.clear() - for (stage <- failedStagesCopy.sortBy(_.jobId)) { - submitStage(stage) + if (failedStages.size > 0) { + // Failed stages may be removed by job cancellation, so failed might be empty even if + // the ResubmitFailedStages event has been scheduled. + logInfo("Resubmitting failed stages") + clearCacheLocs() + val failedStagesCopy = failedStages.toArray + failedStages.clear() + for (stage <- failedStagesCopy.sortBy(_.jobId)) { + submitStage(stage) + } } + submitWaitingStages() } /** * Check for waiting or failed stages which are now eligible for resubmission. * Ordinarily run on every iteration of the event loop. */ - private[scheduler] def submitWaitingStages() { + private def submitWaitingStages() { // TODO: We might want to run this less often, when we are sure that something has become // runnable that wasn't before. logTrace("Checking for newly runnable parent stages") @@ -676,8 +581,12 @@ class DAGScheduler( } } catch { case e: Exception => - jobResult = JobFailed(e, job.finalStage.id) + jobResult = JobFailed(e) job.listener.jobFailed(e) + case oom: OutOfMemoryError => + val exception = new SparkException("Local job aborted due to out of memory error", oom) + jobResult = JobFailed(exception) + job.listener.jobFailed(exception) } finally { val s = job.finalStage stageIdToJobIds -= s.id // clean up data structures that were populated for a local job, @@ -702,6 +611,102 @@ 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 jobIds = activeInGroup.map(_.jobId) + jobIds.foreach(handleJobCancellation(_, "part of cancelled job group %s".format(groupId))) + submitWaitingStages() + } + + private[scheduler] def handleBeginEvent(task: Task[_], taskInfo: TaskInfo) { + for (stage <- stageIdToStage.get(task.stageId); stageInfo <- stageToInfos.get(stage)) { + if (taskInfo.serializedSize > DAGScheduler.TASK_SIZE_TO_WARN * 1024 && + !stageInfo.emittedTaskSizeWarning) { + stageInfo.emittedTaskSizeWarning = true + logWarning(("Stage %d (%s) contains a task of very large " + + "size (%d KB). The maximum recommended task size is %d KB.").format( + task.stageId, stageInfo.name, taskInfo.serializedSize / 1024, + DAGScheduler.TASK_SIZE_TO_WARN)) + } + } + listenerBus.post(SparkListenerTaskStart(task.stageId, taskInfo)) + submitWaitingStages() + } + + private[scheduler] def handleTaskSetFailed(taskSet: TaskSet, reason: String) { + stageIdToStage.get(taskSet.stageId).foreach {abortStage(_, reason) } + submitWaitingStages() + } + + private[scheduler] def cleanUpAfterSchedulerStop() { + for (job <- activeJobs) { + val error = new SparkException("Job cancelled because SparkContext was shut down") + job.listener.jobFailed(error) + // Tell the listeners that all of the running stages have ended. Don't bother + // 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 => + val info = stageToInfos(stage) + info.stageFailed(stageFailedMessage) + listenerBus.post(SparkListenerStageCompleted(info)) + } + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) + } + } + + private[scheduler] def handleGetTaskResult(taskInfo: TaskInfo) { + listenerBus.post(SparkListenerTaskGettingResult(taskInfo)) + submitWaitingStages() + } + + private[scheduler] def handleJobSubmitted(jobId: Int, + finalRDD: RDD[_], + func: (TaskContext, Iterator[_]) => _, + partitions: Array[Int], + allowLocal: Boolean, + callSite: CallSite, + listener: JobListener, + properties: Properties = null) + { + var finalStage: Stage = 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) + } catch { + case e: Exception => + logWarning("Creating new stage failed due to exception - job: " + jobId, e) + listener.jobFailed(e) + return + } + if (finalStage != null) { + val job = new ActiveJob(jobId, finalStage, func, partitions, callSite, listener, properties) + clearCacheLocs() + logInfo("Got job %s (%s) with %d output partitions (allowLocal=%s)".format( + job.jobId, callSite.short, partitions.length, allowLocal)) + logInfo("Final stage: " + finalStage + "(" + finalStage.name + ")") + logInfo("Parents of final stage: " + finalStage.parents) + logInfo("Missing parents: " + getMissingParentStages(finalStage)) + if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { + // Compute very short actions like first() or take() with no parent stages locally. + listenerBus.post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) + runLocally(job) + } else { + jobIdToActiveJob(jobId) = job + activeJobs += job + resultStageToJob(finalStage) = job + listenerBus.post(SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, + properties)) + submitStage(finalStage) + } + } + submitWaitingStages() + } + /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { val jobId = activeJobForStage(stage) @@ -779,7 +784,7 @@ class DAGScheduler( logDebug("New pending tasks: " + myPending) taskScheduler.submitTasks( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties)) - stageToInfos(stage).submissionTime = Some(System.currentTimeMillis()) + stageToInfos(stage).submissionTime = Some(clock.getTime()) } else { logDebug("Stage " + stage + " is actually done; %b %d %d".format( stage.isAvailable, stage.numAvailableOutputs, stage.numPartitions)) @@ -791,9 +796,12 @@ class DAGScheduler( * Responds to a task finishing. This is called inside the event loop so it assumes that it can * modify the scheduler's internal state. Use taskEnded() to post a task end event from outside. */ - private def handleTaskCompletion(event: CompletionEvent) { + private[scheduler] def handleTaskCompletion(event: CompletionEvent) { val task = event.task - + val stageId = task.stageId + val taskType = Utils.getFormattedClassName(task) + listenerBus.post(SparkListenerTaskEnd(stageId, taskType, event.reason, event.taskInfo, + event.taskMetrics)) if (!stageIdToStage.contains(task.stageId)) { // Skip all the actions if the stage has been cancelled. return @@ -802,11 +810,11 @@ class DAGScheduler( def markStageAsFinished(stage: Stage) = { val serviceTime = stageToInfos(stage).submissionTime match { - case Some(t) => "%.03f".format((System.currentTimeMillis() - t) / 1000.0) + case Some(t) => "%.03f".format((clock.getTime() - t) / 1000.0) case _ => "Unknown" } logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) - stageToInfos(stage).completionTime = Some(System.currentTimeMillis()) + stageToInfos(stage).completionTime = Some(clock.getTime()) listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) runningStages -= stage } @@ -826,11 +834,8 @@ class DAGScheduler( job.numFinished += 1 // If the whole job has finished, remove it if (job.numFinished == job.numPartitions) { - jobIdToActiveJob -= stage.jobId - activeJobs -= job - resultStageToJob -= stage markStageAsFinished(stage) - jobIdToStageIdsRemove(job.jobId) + cleanupStateForJobAndIndependentStages(job, Some(stage)) listenerBus.post(SparkListenerJobEnd(job.jobId, JobSucceeded)) } job.listener.taskSucceeded(rt.outputId, event.result) @@ -939,6 +944,7 @@ class DAGScheduler( // Unrecognized failure - also do nothing. If the task fails repeatedly, the TaskScheduler // will abort the job. } + submitWaitingStages() } /** @@ -948,7 +954,7 @@ class DAGScheduler( * Optionally the epoch during which the failure was caught can be passed to avoid allowing * stray fetch failures from possibly retriggering the detection of a node as lost. */ - private def handleExecutorLost(execId: String, maybeEpoch: Option[Long] = None) { + private[scheduler] def handleExecutorLost(execId: String, maybeEpoch: Option[Long] = None) { val currentEpoch = maybeEpoch.getOrElse(mapOutputTracker.getEpoch) if (!failedEpoch.contains(execId) || failedEpoch(execId) < currentEpoch) { failedEpoch(execId) = currentEpoch @@ -968,58 +974,108 @@ class DAGScheduler( logDebug("Additional executor lost message for " + execId + "(epoch " + currentEpoch + ")") } + submitWaitingStages() } - private def handleExecutorAdded(execId: String, host: String) { + private[scheduler] def handleExecutorAdded(execId: String, host: String) { // remove from failedEpoch(execId) ? if (failedEpoch.contains(execId)) { logInfo("Host added was in lost list earlier: " + host) failedEpoch -= execId } + submitWaitingStages() } - private def handleJobCancellation(jobId: Int) { + private[scheduler] def handleStageCancellation(stageId: Int) { + if (stageIdToJobIds.contains(stageId)) { + val jobsThatUseStage: Array[Int] = stageIdToJobIds(stageId).toArray + jobsThatUseStage.foreach(jobId => { + handleJobCancellation(jobId, "because Stage %s was cancelled".format(stageId)) + }) + } else { + logInfo("No active jobs to kill for Stage " + stageId) + } + submitWaitingStages() + } + + private[scheduler] def handleJobCancellation(jobId: Int, reason: String = "") { if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to cancel unregistered job " + jobId) } else { - val independentStages = removeJobAndIndependentStages(jobId) - independentStages.foreach(taskScheduler.cancelTasks) - val error = new SparkException("Job %d cancelled".format(jobId)) - val job = jobIdToActiveJob(jobId) - job.listener.jobFailed(error) - jobIdToStageIds -= jobId - activeJobs -= job - jobIdToActiveJob -= jobId - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, job.finalStage.id))) + failJobAndIndependentStages(jobIdToActiveJob(jobId), + "Job %d cancelled %s".format(jobId, reason), None) } + submitWaitingStages() } /** * 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. */ - private def abortStage(failedStage: Stage, reason: String) { + private[scheduler] def abortStage(failedStage: Stage, reason: String) { if (!stageIdToStage.contains(failedStage.id)) { // Skip all the actions if the stage has been removed. return } val dependentStages = resultStageToJob.keys.filter(x => stageDependsOn(x, failedStage)).toSeq - stageToInfos(failedStage).completionTime = Some(System.currentTimeMillis()) + stageToInfos(failedStage).completionTime = Some(clock.getTime()) for (resultStage <- dependentStages) { val job = resultStageToJob(resultStage) - val error = new SparkException("Job aborted: " + reason) - job.listener.jobFailed(error) - jobIdToStageIdsRemove(job.jobId) - jobIdToActiveJob -= resultStage.jobId - activeJobs -= job - resultStageToJob -= resultStage - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, failedStage.id))) + failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason", + Some(resultStage)) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") } } + /** + * Fails a job and all stages that are only used by that job, and cleans up relevant state. + * + * @param resultStage The result stage for the job, if known. Used to cleanup state for the job + * slightly more efficiently than when not specified. + */ + private def failJobAndIndependentStages(job: ActiveJob, failureReason: String, + resultStage: Option[Stage]) { + val error = new SparkException(failureReason) + job.listener.jobFailed(error) + + val shouldInterruptThread = + if (job.properties == null) false + else job.properties.getProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "false").toBoolean + + // Cancel all independent, running stages. + val stages = jobIdToStageIds(job.jobId) + if (stages.isEmpty) { + logError("No stages registered for job " + job.jobId) + } + stages.foreach { stageId => + val jobsForStage = stageIdToJobIds.get(stageId) + if (jobsForStage.isEmpty || !jobsForStage.get.contains(job.jobId)) { + logError( + "Job %d not registered for stage %d even though that stage was registered for the job" + .format(job.jobId, stageId)) + } else if (jobsForStage.get.size == 1) { + if (!stageIdToStage.contains(stageId)) { + logError("Missing Stage for stage with id $stageId") + } else { + // This is the only job that uses this stage, so fail the stage if it is running. + val stage = stageIdToStage(stageId) + if (runningStages.contains(stage)) { + taskScheduler.cancelTasks(stageId, shouldInterruptThread) + val stageInfo = stageToInfos(stage) + stageInfo.stageFailed(failureReason) + listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) + } + } + } + } + + cleanupStateForJobAndIndependentStages(job, resultStage) + + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) + } + /** * Return true if one of stage's ancestors is target. */ @@ -1034,7 +1090,7 @@ class DAGScheduler( visitedRdds += rdd for (dep <- rdd.dependencies) { dep match { - case shufDep: ShuffleDependency[_,_] => + case shufDep: ShuffleDependency[_, _, _] => val mapStage = getShuffleMapStage(shufDep, stage.jobId) if (!mapStage.isAvailable) { visitedStages += mapStage @@ -1084,27 +1140,86 @@ class DAGScheduler( Nil } - private def cleanup(cleanupTime: Long) { - Map( - "stageIdToStage" -> stageIdToStage, - "shuffleToMapStage" -> shuffleToMapStage, - "pendingTasks" -> pendingTasks, - "stageToInfos" -> stageToInfos, - "jobIdToStageIds" -> jobIdToStageIds, - "stageIdToJobIds" -> stageIdToJobIds). - foreach { case (s, t) => - val sizeBefore = t.size - t.clearOldValues(cleanupTime) - logInfo("%s %d --> %d".format(s, sizeBefore, t.size)) - } + def stop() { + logInfo("Stopping DAGScheduler") + dagSchedulerActorSupervisor ! PoisonPill + taskScheduler.stop() } +} - def stop() { - if (eventProcessActor != null) { - eventProcessActor ! StopDAGScheduler +private[scheduler] class DAGSchedulerActorSupervisor(dagScheduler: DAGScheduler) + extends Actor with Logging { + + override val supervisorStrategy = + OneForOneStrategy() { + case x: Exception => + logError("eventProcesserActor failed due to the error %s; shutting down SparkContext" + .format(x.getMessage)) + dagScheduler.doCancelAllJobs() + dagScheduler.sc.stop() + Stop } - metadataCleaner.cancel() - taskScheduler.stop() + + def receive = { + case p: Props => sender ! context.actorOf(p) + case _ => logWarning("received unknown message in DAGSchedulerActorSupervisor") + } +} + +private[scheduler] class DAGSchedulerEventProcessActor(dagScheduler: DAGScheduler) + extends Actor with Logging { + + override def preStart() { + // set DAGScheduler for taskScheduler to ensure eventProcessActor is always + // valid when the messages arrive + dagScheduler.taskScheduler.setDAGScheduler(dagScheduler) + } + + /** + * The main event loop of the DAG scheduler. + */ + def receive = { + case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) => + dagScheduler.handleJobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, + listener, properties) + + case StageCancelled(stageId) => + dagScheduler.handleStageCancellation(stageId) + + case JobCancelled(jobId) => + dagScheduler.handleJobCancellation(jobId) + + case JobGroupCancelled(groupId) => + dagScheduler.handleJobGroupCancelled(groupId) + + case AllJobsCancelled => + dagScheduler.doCancelAllJobs() + + case ExecutorAdded(execId, host) => + dagScheduler.handleExecutorAdded(execId, host) + + case ExecutorLost(execId) => + dagScheduler.handleExecutorLost(execId) + + case BeginEvent(task, taskInfo) => + dagScheduler.handleBeginEvent(task, taskInfo) + + case GettingResultEvent(taskInfo) => + dagScheduler.handleGetTaskResult(taskInfo) + + case completion @ CompletionEvent(task, reason, _, _, taskInfo, taskMetrics) => + dagScheduler.handleTaskCompletion(completion) + + case TaskSetFailed(taskSet, reason) => + dagScheduler.handleTaskSetFailed(taskSet, reason) + + case ResubmitFailedStages => + dagScheduler.resubmitFailedStages() + } + + override def postStop() { + // Cancel any active jobs in postStop hook + dagScheduler.cleanUpAfterSchedulerStop() } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 04c53d468465a..2b6f7e4205c32 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -20,10 +20,12 @@ package org.apache.spark.scheduler import java.util.Properties import scala.collection.mutable.Map +import scala.language.existentials import org.apache.spark._ import org.apache.spark.executor.TaskMetrics import org.apache.spark.rdd.RDD +import org.apache.spark.util.CallSite /** * Types of events that can be handled by the DAGScheduler. The DAGScheduler uses an event queue @@ -39,11 +41,13 @@ private[scheduler] case class JobSubmitted( func: (TaskContext, Iterator[_]) => _, partitions: Array[Int], allowLocal: Boolean, - callSite: String, + callSite: CallSite, listener: JobListener, properties: Properties = null) extends DAGSchedulerEvent +private[scheduler] case class StageCancelled(stageId: Int) extends DAGSchedulerEvent + private[scheduler] case class JobCancelled(jobId: Int) extends DAGSchedulerEvent private[scheduler] case class JobGroupCancelled(groupId: String) extends DAGSchedulerEvent @@ -54,7 +58,7 @@ private[scheduler] case class BeginEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent private[scheduler] -case class GettingResultEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent +case class GettingResultEvent(taskInfo: TaskInfo) extends DAGSchedulerEvent private[scheduler] case class CompletionEvent( task: Task[_], @@ -73,5 +77,3 @@ private[scheduler] case class TaskSetFailed(taskSet: TaskSet, reason: String) extends DAGSchedulerEvent private[scheduler] case object ResubmitFailedStages extends DAGSchedulerEvent - -private[scheduler] case object StopDAGScheduler extends DAGSchedulerEvent 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 217f8825c2ae9..a90b0d475c04e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -17,11 +17,19 @@ package org.apache.spark.scheduler +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.permission.FsPermission +import org.json4s.JsonAST.JValue import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.{JsonProtocol, FileLogger} +import org.apache.spark.util.{FileLogger, JsonProtocol} /** * A SparkListener that logs events to persistent storage. @@ -33,36 +41,54 @@ import org.apache.spark.util.{JsonProtocol, FileLogger} * spark.eventLog.dir - Path to the directory in which events are logged. * spark.eventLog.buffer.kb - Buffer size to use when writing to output streams */ -private[spark] class EventLoggingListener(appName: String, conf: SparkConf) +private[spark] class EventLoggingListener( + appName: String, + sparkConf: SparkConf, + hadoopConf: Configuration = SparkHadoopUtil.get.newConfiguration()) extends SparkListener with Logging { - private val shouldCompress = conf.getBoolean("spark.eventLog.compress", false) - private val shouldOverwrite = conf.getBoolean("spark.eventLog.overwrite", false) - private val outputBufferSize = conf.getInt("spark.eventLog.buffer.kb", 100) * 1024 - private val logBaseDir = conf.get("spark.eventLog.dir", "/tmp/spark-events").stripSuffix("/") + import EventLoggingListener._ + + 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 logBaseDir = sparkConf.get("spark.eventLog.dir", DEFAULT_LOG_DIR).stripSuffix("/") private val name = appName.replaceAll("[ :/]", "-").toLowerCase + "-" + System.currentTimeMillis val logDir = logBaseDir + "/" + name - private val logger = - new FileLogger(logDir, conf, outputBufferSize, shouldCompress, shouldOverwrite) + protected val logger = new FileLogger(logDir, sparkConf, hadoopConf, outputBufferSize, + shouldCompress, shouldOverwrite, Some(LOG_FILE_PERMISSIONS)) - // Information needed to replay the events logged by this listener later - val info = { - val compressionCodec = if (shouldCompress) { - Some(conf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC)) - } else None - EventLoggingInfo(logDir, compressionCodec) - } + // For testing. Keep track of all JSON serialized events that have been logged. + private[scheduler] val loggedEvents = new ArrayBuffer[JValue] - logInfo("Logging events to %s".format(logDir)) + /** + * Begin logging events. + * If compression is used, log a file that indicates which compression library is used. + */ + def start() { + logger.start() + logInfo("Logging events to %s".format(logDir)) + if (shouldCompress) { + val codec = + sparkConf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC) + logger.newFile(COMPRESSION_CODEC_PREFIX + codec) + } + logger.newFile(SPARK_VERSION_PREFIX + SparkContext.SPARK_VERSION) + logger.newFile(LOG_PREFIX + logger.fileIndex) + } - /** Log the event as JSON */ + /** Log the event as JSON. */ private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false) { - val eventJson = compact(render(JsonProtocol.sparkEventToJson(event))) - logger.logLine(eventJson) + val eventJson = JsonProtocol.sparkEventToJson(event) + logger.logLine(compact(render(eventJson))) if (flushLogger) { logger.flush() } + if (testing) { + loggedEvents += eventJson + } } // Events that do not trigger a flush @@ -90,9 +116,120 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) logEvent(event, flushLogger = true) override def onUnpersistRDD(event: SparkListenerUnpersistRDD) = logEvent(event, flushLogger = true) + override def onApplicationStart(event: SparkListenerApplicationStart) = + logEvent(event, flushLogger = true) + override def onApplicationEnd(event: SparkListenerApplicationEnd) = + logEvent(event, flushLogger = true) + + /** + * Stop logging events. + * In addition, create an empty special file to indicate application completion. + */ + def stop() = { + logger.newFile(APPLICATION_COMPLETE) + logger.stop() + } +} + +private[spark] object EventLoggingListener extends Logging { + val DEFAULT_LOG_DIR = "/tmp/spark-events" + val LOG_PREFIX = "EVENT_LOG_" + val SPARK_VERSION_PREFIX = "SPARK_VERSION_" + val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" + val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" + val LOG_FILE_PERMISSIONS = FsPermission.createImmutable(Integer.parseInt("770", 8).toShort) + + // A cache for compression codecs to avoid creating the same codec many times + private val codecMap = new mutable.HashMap[String, CompressionCodec] + + def isEventLogFile(fileName: String): Boolean = { + fileName.startsWith(LOG_PREFIX) + } + + def isSparkVersionFile(fileName: String): Boolean = { + fileName.startsWith(SPARK_VERSION_PREFIX) + } - def stop() = logger.stop() + def isCompressionCodecFile(fileName: String): Boolean = { + fileName.startsWith(COMPRESSION_CODEC_PREFIX) + } + + def isApplicationCompleteFile(fileName: String): Boolean = { + fileName == APPLICATION_COMPLETE + } + + def parseSparkVersion(fileName: String): String = { + if (isSparkVersionFile(fileName)) { + fileName.replaceAll(SPARK_VERSION_PREFIX, "") + } else "" + } + + def parseCompressionCodec(fileName: String): String = { + if (isCompressionCodecFile(fileName)) { + fileName.replaceAll(COMPRESSION_CODEC_PREFIX, "") + } else "" + } + + /** + * Parse the event logging information associated with the logs in the given directory. + * + * Specifically, this looks for event log files, the Spark version file, the compression + * codec file (if event logs are compressed), and the application completion file (if the + * application has run to completion). + */ + def parseLoggingInfo(logDir: Path, fileSystem: FileSystem): EventLoggingInfo = { + try { + val fileStatuses = fileSystem.listStatus(logDir) + val filePaths = + if (fileStatuses != null) { + fileStatuses.filter(!_.isDir).map(_.getPath).toSeq + } else { + Seq[Path]() + } + if (filePaths.isEmpty) { + logWarning("No files found in logging directory %s".format(logDir)) + } + EventLoggingInfo( + logPaths = filePaths.filter { path => isEventLogFile(path.getName) }, + sparkVersion = filePaths + .find { path => isSparkVersionFile(path.getName) } + .map { path => parseSparkVersion(path.getName) } + .getOrElse(""), + compressionCodec = filePaths + .find { path => isCompressionCodecFile(path.getName) } + .map { path => + val codec = EventLoggingListener.parseCompressionCodec(path.getName) + val conf = new SparkConf + conf.set("spark.io.compression.codec", codec) + codecMap.getOrElseUpdate(codec, CompressionCodec.createCodec(conf)) + }, + applicationComplete = filePaths.exists { path => isApplicationCompleteFile(path.getName) } + ) + } catch { + case e: Exception => + logError("Exception in parsing logging info from directory %s".format(logDir), e) + EventLoggingInfo.empty + } + } + + /** + * Parse the event logging information associated with the logs in the given directory. + */ + def parseLoggingInfo(logDir: String, fileSystem: FileSystem): EventLoggingInfo = { + parseLoggingInfo(new Path(logDir), fileSystem) + } } -// If compression is not enabled, compressionCodec is None -private[spark] case class EventLoggingInfo(logDir: String, compressionCodec: Option[String]) + +/** + * Information needed to process the event logs associated with an application. + */ +private[spark] case class EventLoggingInfo( + logPaths: Seq[Path], + sparkVersion: String, + compressionCodec: Option[CompressionCodec], + applicationComplete: Boolean = false) + +private[spark] object EventLoggingInfo { + def empty = EventLoggingInfo(Seq[Path](), "", None, applicationComplete = false) +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala index 5555585c8b4cd..bac37bfdaa23f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -27,11 +27,14 @@ import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.util.ReflectionUtils import org.apache.spark.Logging +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.SparkHadoopUtil /** + * :: DeveloperApi :: * Parses and holds information about inputFormat (and files) specified as a parameter. */ +@DeveloperApi class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_], val path: String) extends Logging { @@ -164,8 +167,7 @@ object InputFormatInfo { PS: I know the wording here is weird, hopefully it makes some sense ! */ - def computePreferredLocations(formats: Seq[InputFormatInfo]): HashMap[String, HashSet[SplitInfo]] - = { + def computePreferredLocations(formats: Seq[InputFormatInfo]): Map[String, Set[SplitInfo]] = { val nodeToSplit = new HashMap[String, HashSet[SplitInfo]] for (inputSplit <- formats) { @@ -178,6 +180,6 @@ object InputFormatInfo { } } - nodeToSplit + nodeToSplit.mapValues(_.toSet).toMap } } 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 5cecf9416b32c..a1e21cad48b9b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -25,9 +25,11 @@ import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.HashMap import org.apache.spark._ +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics /** + * :: DeveloperApi :: * A logger class to record runtime information for jobs in Spark. This class outputs one log file * for each Spark job, containing tasks start/stop and shuffle information. JobLogger is a subclass * of SparkListener, use addSparkListener to add JobLogger to a SparkContext after the SparkContext @@ -38,7 +40,7 @@ import org.apache.spark.executor.TaskMetrics * to log application information as SparkListenerEvents. To enable this functionality, set * spark.eventLog.enabled to true. */ - +@DeveloperApi @deprecated("Log application information by setting spark.eventLog.enabled.", "1.0.0") class JobLogger(val user: String, val logDirName: String) extends SparkListener with Logging { @@ -191,7 +193,11 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener */ override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { val stageId = stageCompleted.stageInfo.stageId - stageLogInfo(stageId, "STAGE_ID=%d STATUS=COMPLETED".format(stageId)) + if (stageCompleted.stageInfo.failureReason.isEmpty) { + stageLogInfo(stageId, s"STAGE_ID=$stageId STATUS=COMPLETED") + } else { + stageLogInfo(stageId, s"STAGE_ID=$stageId STATUS=FAILED") + } } /** @@ -201,7 +207,7 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val taskInfo = taskEnd.taskInfo var taskStatus = "TASK_TYPE=%s".format(taskEnd.taskType) - val taskMetrics = if (taskEnd.taskMetrics != null) taskEnd.taskMetrics else TaskMetrics.empty() + val taskMetrics = if (taskEnd.taskMetrics != null) taskEnd.taskMetrics else TaskMetrics.empty taskEnd.reason match { case Success => taskStatus += " STATUS=SUCCESS" recordTaskMetrics(taskEnd.stageId, taskStatus, taskInfo, taskMetrics) @@ -227,7 +233,7 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener var info = "JOB_ID=" + jobId jobEnd.jobResult match { case JobSucceeded => info += " STATUS=SUCCESS" - case JobFailed(exception, _) => + case JobFailed(exception) => info += " STATUS=FAILED REASON=" exception.getMessage.split("\\s+").foreach(info += _ + "_") case _ => diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index 3cf4e3077e4a4..4cd6cbe189aab 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -17,12 +17,17 @@ package org.apache.spark.scheduler +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A result of a job in the DAGScheduler. */ -private[spark] sealed trait JobResult +@DeveloperApi +sealed trait JobResult -private[spark] case object JobSucceeded extends JobResult +@DeveloperApi +case object JobSucceeded extends JobResult -// A failed stage ID of -1 means there is not a particular stage that caused the failure -private[spark] case class JobFailed(exception: Exception, failedStageId: Int) extends JobResult +@DeveloperApi +private[spark] case class JobFailed(exception: Exception) extends JobResult 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 8007b5418741e..e9bfee2248e5b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala @@ -64,7 +64,7 @@ private[spark] class JobWaiter[T]( override def jobFailed(exception: Exception): Unit = synchronized { _jobFinished = true - jobResult = JobFailed(exception, -1) + jobResult = JobFailed(exception) this.notifyAll() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index 353a48661b0f7..36a6e6338faa6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -1,101 +1,141 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF 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 java.util.concurrent.LinkedBlockingQueue - -import org.apache.spark.Logging - -/** - * Asynchronously passes SparkListenerEvents to registered SparkListeners. - * - * Until start() is called, all posted events are only buffered. Only after this listener bus - * has started will events be actually propagated to all attached listeners. This listener bus - * is stopped when it receives a SparkListenerShutdown event, which is posted using stop(). - */ -private[spark] class LiveListenerBus extends SparkListenerBus with Logging { - - /* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than - * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ - private val EVENT_QUEUE_CAPACITY = 10000 - private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) - private var queueFullErrorMessageLogged = false - private var started = false - - /** - * Start sending events to attached listeners. - * - * This first sends out all buffered events posted before this listener bus has started, then - * listens for any additional events asynchronously while the listener bus is still running. - * This should only be called once. - */ - def start() { - if (started) { - throw new IllegalStateException("Listener bus already started!") - } - started = true - new Thread("SparkListenerBus") { - setDaemon(true) - override def run() { - while (true) { - val event = eventQueue.take - if (event == SparkListenerShutdown) { - // Get out of the while loop and shutdown the daemon thread - return - } - postToAll(event) - } - } - }.start() - } - - def post(event: SparkListenerEvent) { - val eventAdded = eventQueue.offer(event) - if (!eventAdded && !queueFullErrorMessageLogged) { - logError("Dropping SparkListenerEvent because no remaining room in event queue. " + - "This likely means one of the SparkListeners is too slow and cannot keep up with the " + - "rate at which tasks are being started by the scheduler.") - queueFullErrorMessageLogged = true - } - } - - /** - * Waits until there are no more events in the queue, or until the specified time has elapsed. - * Used for testing only. Returns true if the queue has emptied and false is the specified time - * elapsed before the queue emptied. - */ - def waitUntilEmpty(timeoutMillis: Int): Boolean = { - val finishTime = System.currentTimeMillis + timeoutMillis - while (!eventQueue.isEmpty) { - if (System.currentTimeMillis > finishTime) { - return false - } - /* Sleep rather than using wait/notify, because this is used only for testing and wait/notify - * add overhead in the general case. */ - Thread.sleep(10) - } - true - } - - def stop() { - if (!started) { - throw new IllegalStateException("Attempted to stop a listener bus that has not yet started!") - } - post(SparkListenerShutdown) - } -} +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 java.util.concurrent.{LinkedBlockingQueue, Semaphore} + +import org.apache.spark.Logging +import org.apache.spark.util.Utils + +/** + * Asynchronously passes SparkListenerEvents to registered SparkListeners. + * + * Until start() is called, all posted events are only buffered. Only after this listener bus + * has started will events be actually propagated to all attached listeners. This listener bus + * is stopped when it receives a SparkListenerShutdown event, which is posted using stop(). + */ +private[spark] class LiveListenerBus extends SparkListenerBus with Logging { + + /* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than + * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ + private val EVENT_QUEUE_CAPACITY = 10000 + private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) + private var queueFullErrorMessageLogged = false + private var started = false + + // A counter that represents the number of events produced and consumed in the queue + private val eventLock = new Semaphore(0) + + private val listenerThread = new Thread("SparkListenerBus") { + setDaemon(true) + override def run(): Unit = Utils.logUncaughtExceptions { + while (true) { + eventLock.acquire() + // Atomically remove and process this event + LiveListenerBus.this.synchronized { + val event = eventQueue.poll + if (event == SparkListenerShutdown) { + // Get out of the while loop and shutdown the daemon thread + return + } + Option(event).foreach(postToAll) + } + } + } + } + + /** + * Start sending events to attached listeners. + * + * This first sends out all buffered events posted before this listener bus has started, then + * listens for any additional events asynchronously while the listener bus is still running. + * This should only be called once. + */ + def start() { + if (started) { + throw new IllegalStateException("Listener bus already started!") + } + listenerThread.start() + started = true + } + + def post(event: SparkListenerEvent) { + val eventAdded = eventQueue.offer(event) + if (eventAdded) { + eventLock.release() + } else { + logQueueFullErrorMessage() + } + } + + /** + * For testing only. Wait until there are no more events in the queue, or until the specified + * time has elapsed. Return true if the queue has emptied and false is the specified time + * elapsed before the queue emptied. + */ + def waitUntilEmpty(timeoutMillis: Int): Boolean = { + val finishTime = System.currentTimeMillis + timeoutMillis + while (!queueIsEmpty) { + if (System.currentTimeMillis > finishTime) { + return false + } + /* Sleep rather than using wait/notify, because this is used only for testing and + * wait/notify add overhead in the general case. */ + Thread.sleep(10) + } + true + } + + /** + * For testing only. Return whether the listener daemon thread is still alive. + */ + def listenerThreadIsAlive: Boolean = synchronized { listenerThread.isAlive } + + /** + * Return whether the event queue is empty. + * + * The use of synchronized here guarantees that all events that once belonged to this queue + * have already been processed by all attached listeners, if this returns true. + */ + def queueIsEmpty: Boolean = synchronized { eventQueue.isEmpty } + + /** + * Log an error message to indicate that the event queue is full. Do this only once. + */ + private def logQueueFullErrorMessage(): Unit = { + if (!queueFullErrorMessageLogged) { + if (listenerThread.isAlive) { + logError("Dropping SparkListenerEvent because no remaining room in event queue. " + + "This likely means one of the SparkListeners is too slow and cannot keep up with" + + "the rate at which tasks are being started by the scheduler.") + } else { + logError("SparkListenerBus thread is dead! This means SparkListenerEvents have not" + + "been (and will no longer be) propagated to listeners for some time.") + } + queueFullErrorMessageLogged = true + } + } + + def stop() { + if (!started) { + throw new IllegalStateException("Attempted to stop a listener bus that has not yet started!") + } + post(SparkListenerShutdown) + listenerThread.join() + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala index 187672c4e19e7..174b73221afc0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala @@ -17,8 +17,10 @@ package org.apache.spark.scheduler +import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue} + +import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap import org.apache.spark.Logging import org.apache.spark.scheduler.SchedulingMode.SchedulingMode @@ -35,18 +37,15 @@ private[spark] class Pool( extends Schedulable with Logging { - var schedulableQueue = new ArrayBuffer[Schedulable] - var schedulableNameToSchedulable = new HashMap[String, Schedulable] - + val schedulableQueue = new ConcurrentLinkedQueue[Schedulable] + val schedulableNameToSchedulable = new ConcurrentHashMap[String, Schedulable] var weight = initWeight var minShare = initMinShare var runningTasks = 0 - var priority = 0 // A pool's stage id is used to break the tie in scheduling. var stageId = -1 - var name = poolName var parent: Pool = null @@ -60,19 +59,20 @@ private[spark] class Pool( } override def addSchedulable(schedulable: Schedulable) { - schedulableQueue += schedulable - schedulableNameToSchedulable(schedulable.name) = schedulable + require(schedulable != null) + schedulableQueue.add(schedulable) + schedulableNameToSchedulable.put(schedulable.name, schedulable) schedulable.parent = this } override def removeSchedulable(schedulable: Schedulable) { - schedulableQueue -= schedulable - schedulableNameToSchedulable -= schedulable.name + schedulableQueue.remove(schedulable) + schedulableNameToSchedulable.remove(schedulable.name) } override def getSchedulableByName(schedulableName: String): Schedulable = { - if (schedulableNameToSchedulable.contains(schedulableName)) { - return schedulableNameToSchedulable(schedulableName) + if (schedulableNameToSchedulable.containsKey(schedulableName)) { + return schedulableNameToSchedulable.get(schedulableName) } for (schedulable <- schedulableQueue) { val sched = schedulable.getSchedulableByName(schedulableName) @@ -95,11 +95,12 @@ private[spark] class Pool( shouldRevive } - override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { + override def getSortedTaskSetQueue: ArrayBuffer[TaskSetManager] = { var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] - val sortedSchedulableQueue = schedulableQueue.sortWith(taskSetSchedulingAlgorithm.comparator) + val sortedSchedulableQueue = + schedulableQueue.toSeq.sortWith(taskSetSchedulingAlgorithm.comparator) for (schedulable <- sortedSchedulableQueue) { - sortedTaskSetQueue ++= schedulable.getSortedTaskSetQueue() + sortedTaskSetQueue ++= schedulable.getSortedTaskSetQueue } sortedTaskSetQueue } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index db76178b65501..f89724d4ea196 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -17,72 +17,54 @@ package org.apache.spark.scheduler -import java.io.InputStream -import java.net.URI +import java.io.{BufferedInputStream, InputStream} import scala.io.Source -import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.hadoop.fs.{Path, FileSystem} import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.Logging import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.{JsonProtocol, Utils} +import org.apache.spark.util.JsonProtocol /** - * An EventBus that replays logged events from persisted storage + * A SparkListenerBus that replays logged events from persisted storage. + * + * This assumes the given paths are valid log files, where each line can be deserialized into + * exactly one SparkListenerEvent. */ -private[spark] class ReplayListenerBus(conf: SparkConf) extends SparkListenerBus with Logging { - private val compressed = conf.getBoolean("spark.eventLog.compress", false) +private[spark] class ReplayListenerBus( + logPaths: Seq[Path], + fileSystem: FileSystem, + compressionCodec: Option[CompressionCodec]) + extends SparkListenerBus with Logging { - // Only used if compression is enabled - private lazy val compressionCodec = CompressionCodec.createCodec(conf) + private var replayed = false - /** - * Return a list of paths representing log files in the given directory. - */ - private def getLogFilePaths(logDir: String, fileSystem: FileSystem): Array[Path] = { - val path = new Path(logDir) - if (!fileSystem.exists(path) || !fileSystem.getFileStatus(path).isDir) { - logWarning("Log path provided is not a valid directory: %s".format(logDir)) - return Array[Path]() - } - val logStatus = fileSystem.listStatus(path) - if (logStatus == null || !logStatus.exists(!_.isDir)) { - logWarning("Log path provided contains no log files: %s".format(logDir)) - return Array[Path]() - } - logStatus.filter(!_.isDir).map(_.getPath).sortBy(_.getName) + if (logPaths.length == 0) { + logWarning("Log path provided contains no log files.") } /** * Replay each event in the order maintained in the given logs. + * This should only be called exactly once. */ - def replay(logDir: String): Boolean = { - val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) - val logPaths = getLogFilePaths(logDir, fileSystem) - if (logPaths.length == 0) { - return false - } - + def replay() { + assert(!replayed, "ReplayListenerBus cannot replay events more than once") logPaths.foreach { path => // Keep track of input streams at all levels to close them later // This is necessary because an exception can occur in between stream initializations var fileStream: Option[InputStream] = None var bufferedStream: Option[InputStream] = None var compressStream: Option[InputStream] = None - var currentLine = "" + var currentLine = "" try { - currentLine = "" fileStream = Some(fileSystem.open(path)) - bufferedStream = Some(new FastBufferedInputStream(fileStream.get)) - compressStream = - if (compressed) { - Some(compressionCodec.compressedInputStream(bufferedStream.get)) - } else bufferedStream + bufferedStream = Some(new BufferedInputStream(fileStream.get)) + compressStream = Some(wrapForCompression(bufferedStream.get)) - // Parse each line as an event and post it to all attached listeners + // Parse each line as an event and post the event to all attached listeners val lines = Source.fromInputStream(compressStream.get).getLines() lines.foreach { line => currentLine = line @@ -98,7 +80,11 @@ private[spark] class ReplayListenerBus(conf: SparkConf) extends SparkListenerBus compressStream.foreach(_.close()) } } - fileSystem.close() - true + replayed = true + } + + /** If a compression codec is specified, wrap the given stream in a compression stream. */ + private def wrapForCompression(stream: InputStream): InputStream = { + compressionCodec.map(_.compressedInputStream(stream)).getOrElse(stream) } } 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 3fc6cc9850feb..bbf9f7388b074 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -17,24 +17,22 @@ package org.apache.spark.scheduler +import scala.language.existentials + import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} +import scala.collection.mutable.HashMap + import org.apache.spark._ -import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.RDDCheckpointData -import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} +import org.apache.spark.rdd.{RDD, RDDCheckpointData} private[spark] object ResultTask { // A simple map between the stage id to the serialized byte array of a task. // Served as a cache for task serialization because serialization can be // expensive on the master node if it needs to launch thousands of tasks. - val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]] - - // TODO: This object shouldn't have global variables - val metadataCleaner = new MetadataCleaner( - MetadataCleanerType.RESULT_TASK, serializedInfoCache.clearOldValues, new SparkConf) + private val serializedInfoCache = new HashMap[Int, Array[Byte]] def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = { @@ -58,7 +56,6 @@ private[spark] object ResultTask { def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], (TaskContext, Iterator[_]) => _) = { - val loader = Thread.currentThread.getContextClassLoader val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) val ser = SparkEnv.get.closureSerializer.newInstance() val objIn = ser.deserializeStream(in) @@ -67,6 +64,10 @@ private[spark] object ResultTask { (rdd, func) } + def removeStage(stageId: Int) { + serializedInfoCache.remove(stageId) + } + def clearCache() { synchronized { serializedInfoCache.clear() diff --git a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala index ed24eb6a549dd..a87ef030e69c2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import java.util.concurrent.ConcurrentLinkedQueue + import scala.collection.mutable.ArrayBuffer import org.apache.spark.scheduler.SchedulingMode.SchedulingMode @@ -28,7 +30,7 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode private[spark] trait Schedulable { var parent: Pool // child queues - def schedulableQueue: ArrayBuffer[Schedulable] + def schedulableQueue: ConcurrentLinkedQueue[Schedulable] def schedulingMode: SchedulingMode def weight: Int def minShare: Int @@ -42,5 +44,5 @@ private[spark] trait Schedulable { def getSchedulableByName(name: String): Schedulable def executorLost(executorId: String, host: String): Unit def checkSpeculatableTasks(): Boolean - def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] + def getSortedTaskSetQueue: ArrayBuffer[TaskSetManager] } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala index e4eced383c3a5..6c5827f75e636 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala @@ -23,6 +23,7 @@ import java.util.{NoSuchElementException, Properties} import scala.xml.XML import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.util.Utils /** * An interface to build Schedulable tree @@ -72,7 +73,7 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool, conf: SparkConf) schedulerAllocFile.map { f => new FileInputStream(f) }.getOrElse { - getClass.getClassLoader.getResourceAsStream(DEFAULT_SCHEDULER_FILE) + Utils.getSparkClassLoader.getResourceAsStream(DEFAULT_SCHEDULER_FILE) } } 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 f1924a4573b21..6a6d8e609bc39 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -28,5 +28,6 @@ private[spark] trait SchedulerBackend { def reviveOffers(): Unit def defaultParallelism(): Int - def killTask(taskId: Long, executorId: String): Unit = throw new UnsupportedOperationException + def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = + throw new UnsupportedOperationException } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulingMode.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulingMode.scala index 3832ee7ff6eef..75186b6ba4a41 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulingMode.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulingMode.scala @@ -25,5 +25,5 @@ package org.apache.spark.scheduler object SchedulingMode extends Enumeration { type SchedulingMode = Value - val FAIR,FIFO,NONE = Value + val FAIR, FIFO, NONE = Value } 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 2a9edf4a76b97..859cdc524a581 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -17,31 +17,25 @@ package org.apache.spark.scheduler +import scala.language.existentials + import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.RDDCheckpointData -import org.apache.spark.serializer.Serializer -import org.apache.spark.storage._ -import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} +import org.apache.spark.rdd.{RDD, RDDCheckpointData} +import org.apache.spark.shuffle.ShuffleWriter private[spark] object ShuffleMapTask { // A simple map between the stage id to the serialized byte array of a task. // Served as a cache for task serialization because serialization can be // expensive on the master node if it needs to launch thousands of tasks. - val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]] - - // TODO: This object shouldn't have global variables - val metadataCleaner = new MetadataCleaner( - MetadataCleanerType.SHUFFLE_MAP_TASK, serializedInfoCache.clearOldValues, new SparkConf) + private val serializedInfoCache = new HashMap[Int, Array[Byte]] - def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_,_]): Array[Byte] = { + def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_, _, _]): Array[Byte] = { synchronized { val old = serializedInfoCache.get(stageId).orNull if (old != null) { @@ -60,26 +54,27 @@ private[spark] object ShuffleMapTask { } } - def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], ShuffleDependency[_,_]) = { - synchronized { - val loader = Thread.currentThread.getContextClassLoader - val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) - val ser = SparkEnv.get.closureSerializer.newInstance() - val objIn = ser.deserializeStream(in) - val rdd = objIn.readObject().asInstanceOf[RDD[_]] - val dep = objIn.readObject().asInstanceOf[ShuffleDependency[_,_]] - (rdd, dep) - } + def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], ShuffleDependency[_, _, _]) = { + val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) + val ser = SparkEnv.get.closureSerializer.newInstance() + val objIn = ser.deserializeStream(in) + val rdd = objIn.readObject().asInstanceOf[RDD[_]] + val dep = objIn.readObject().asInstanceOf[ShuffleDependency[_, _, _]] + (rdd, dep) } // Since both the JarSet and FileSet have the same format this is used for both. - def deserializeFileSet(bytes: Array[Byte]) : HashMap[String, Long] = { + def deserializeFileSet(bytes: Array[Byte]): HashMap[String, Long] = { val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) val objIn = new ObjectInputStream(in) val set = objIn.readObject().asInstanceOf[Array[(String, Long)]].toMap HashMap(set.toSeq: _*) } + def removeStage(stageId: Int) { + serializedInfoCache.remove(stageId) + } + def clearCache() { synchronized { serializedInfoCache.clear() @@ -102,7 +97,7 @@ private[spark] object ShuffleMapTask { private[spark] class ShuffleMapTask( stageId: Int, var rdd: RDD[_], - var dep: ShuffleDependency[_,_], + var dep: ShuffleDependency[_, _, _], _partitionId: Int, @transient private var locs: Seq[TaskLocation]) extends Task[MapStatus](stageId, _partitionId) @@ -144,62 +139,22 @@ private[spark] class ShuffleMapTask( } override def runTask(context: TaskContext): MapStatus = { - val numOutputSplits = dep.partitioner.numPartitions metrics = Some(context.taskMetrics) - - val blockManager = SparkEnv.get.blockManager - val shuffleBlockManager = blockManager.shuffleBlockManager - var shuffle: ShuffleWriterGroup = null - var success = false - + var writer: ShuffleWriter[Any, Any] = null try { - // Obtain all the block writers for shuffle blocks. - val ser = Serializer.getSerializer(dep.serializer) - shuffle = shuffleBlockManager.forMapTask(dep.shuffleId, partitionId, numOutputSplits, ser) - - // Write the map output to its associated buckets. + val manager = SparkEnv.get.shuffleManager + writer = manager.getWriter[Any, Any](dep.shuffleHandle, partitionId, context) for (elem <- rdd.iterator(split, context)) { - val pair = elem.asInstanceOf[Product2[Any, Any]] - val bucketId = dep.partitioner.getPartition(pair._1) - shuffle.writers(bucketId).write(pair) - } - - // Commit the writes. Get the size of each bucket block (total block size). - var totalBytes = 0L - var totalTime = 0L - val compressedSizes: Array[Byte] = shuffle.writers.map { writer: BlockObjectWriter => - writer.commit() - writer.close() - val size = writer.fileSegment().length - totalBytes += size - totalTime += writer.timeWriting() - MapOutputTracker.compressSize(size) + writer.write(elem.asInstanceOf[Product2[Any, Any]]) } - - // Update shuffle metrics. - val shuffleMetrics = new ShuffleWriteMetrics - shuffleMetrics.shuffleBytesWritten = totalBytes - shuffleMetrics.shuffleWriteTime = totalTime - metrics.get.shuffleWriteMetrics = Some(shuffleMetrics) - - success = true - new MapStatus(blockManager.blockManagerId, compressedSizes) - } catch { case e: Exception => - // If there is an exception from running the task, revert the partial writes - // and throw the exception upstream to Spark. - if (shuffle != null && shuffle.writers != null) { - for (writer <- shuffle.writers) { - writer.revertPartialWrites() - writer.close() + writer.stop(success = true).get + } catch { + case e: Exception => + if (writer != null) { + writer.stop(success = false) } - } - throw e + throw e } finally { - // Release the writers back to the shuffle block manager. - if (shuffle != null && shuffle.writers != null) { - shuffle.releaseWriters(success) - } - // Execute the callbacks on task completion. context.executeOnCompleteCallbacks() } } 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 d4eb0ac88d8e8..378cf1aaebe7b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -23,21 +23,28 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.{Logging, TaskEndReason} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{Distribution, Utils} +@DeveloperApi sealed trait SparkListenerEvent +@DeveloperApi case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) extends SparkListenerEvent +@DeveloperApi case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent +@DeveloperApi case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent +@DeveloperApi case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent +@DeveloperApi case class SparkListenerTaskEnd( stageId: Int, taskType: String, @@ -46,32 +53,46 @@ case class SparkListenerTaskEnd( taskMetrics: TaskMetrics) extends SparkListenerEvent +@DeveloperApi case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) extends SparkListenerEvent +@DeveloperApi case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent +@DeveloperApi case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) extends SparkListenerEvent +@DeveloperApi case class SparkListenerBlockManagerAdded(blockManagerId: BlockManagerId, maxMem: Long) extends SparkListenerEvent +@DeveloperApi case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) extends SparkListenerEvent +@DeveloperApi case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent +case class SparkListenerApplicationStart(appName: String, time: Long, sparkUser: String) + extends SparkListenerEvent + +case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent + /** An event used in the listener to shutdown the listener daemon thread. */ private[spark] case object SparkListenerShutdown extends SparkListenerEvent /** - * Interface for listening to events from the Spark scheduler. + * :: DeveloperApi :: + * Interface for listening to events from the Spark scheduler. Note that this is an internal + * interface which might change in different Spark releases. */ +@DeveloperApi trait SparkListener { /** - * Called when a stage is completed, with information on the completed stage + * Called when a stage completes successfully or fails, with information on the completed stage. */ def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { } @@ -125,11 +146,23 @@ trait SparkListener { * Called when an RDD is manually unpersisted by the application */ def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) { } + + /** + * Called when the application starts + */ + def onApplicationStart(applicationStart: SparkListenerApplicationStart) { } + + /** + * Called when the application ends + */ + def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { } } /** + * :: DeveloperApi :: * Simple SparkListener that logs a few summary statistics when each stage completes */ +@DeveloperApi class StatsReportListener extends SparkListener with Logging { import org.apache.spark.scheduler.StatsReportListener._ diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 729e120497571..ed9fb24bc8ce8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -20,10 +20,13 @@ package org.apache.spark.scheduler import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import org.apache.spark.Logging +import org.apache.spark.util.Utils + /** * A SparkListenerEvent bus that relays events to its listeners */ -private[spark] trait SparkListenerBus { +private[spark] trait SparkListenerBus extends Logging { // SparkListeners attached to this event bus protected val sparkListeners = new ArrayBuffer[SparkListener] @@ -34,34 +37,53 @@ private[spark] trait SparkListenerBus { } /** - * Post an event to all attached listeners. This does nothing if the event is - * SparkListenerShutdown. + * Post an event to all attached listeners. + * This does nothing if the event is SparkListenerShutdown. */ - protected def postToAll(event: SparkListenerEvent) { + def postToAll(event: SparkListenerEvent) { event match { case stageSubmitted: SparkListenerStageSubmitted => - sparkListeners.foreach(_.onStageSubmitted(stageSubmitted)) + foreachListener(_.onStageSubmitted(stageSubmitted)) case stageCompleted: SparkListenerStageCompleted => - sparkListeners.foreach(_.onStageCompleted(stageCompleted)) + foreachListener(_.onStageCompleted(stageCompleted)) case jobStart: SparkListenerJobStart => - sparkListeners.foreach(_.onJobStart(jobStart)) + foreachListener(_.onJobStart(jobStart)) case jobEnd: SparkListenerJobEnd => - sparkListeners.foreach(_.onJobEnd(jobEnd)) + foreachListener(_.onJobEnd(jobEnd)) case taskStart: SparkListenerTaskStart => - sparkListeners.foreach(_.onTaskStart(taskStart)) + foreachListener(_.onTaskStart(taskStart)) case taskGettingResult: SparkListenerTaskGettingResult => - sparkListeners.foreach(_.onTaskGettingResult(taskGettingResult)) + foreachListener(_.onTaskGettingResult(taskGettingResult)) case taskEnd: SparkListenerTaskEnd => - sparkListeners.foreach(_.onTaskEnd(taskEnd)) + foreachListener(_.onTaskEnd(taskEnd)) case environmentUpdate: SparkListenerEnvironmentUpdate => - sparkListeners.foreach(_.onEnvironmentUpdate(environmentUpdate)) + foreachListener(_.onEnvironmentUpdate(environmentUpdate)) case blockManagerAdded: SparkListenerBlockManagerAdded => - sparkListeners.foreach(_.onBlockManagerAdded(blockManagerAdded)) + foreachListener(_.onBlockManagerAdded(blockManagerAdded)) case blockManagerRemoved: SparkListenerBlockManagerRemoved => - sparkListeners.foreach(_.onBlockManagerRemoved(blockManagerRemoved)) + foreachListener(_.onBlockManagerRemoved(blockManagerRemoved)) case unpersistRDD: SparkListenerUnpersistRDD => - sparkListeners.foreach(_.onUnpersistRDD(unpersistRDD)) + foreachListener(_.onUnpersistRDD(unpersistRDD)) + case applicationStart: SparkListenerApplicationStart => + foreachListener(_.onApplicationStart(applicationStart)) + case applicationEnd: SparkListenerApplicationEnd => + foreachListener(_.onApplicationEnd(applicationEnd)) case SparkListenerShutdown => } } + + /** + * Apply the given function to all attached listeners, catching and logging any exception. + */ + private def foreachListener(f: SparkListener => Unit): Unit = { + sparkListeners.foreach { listener => + try { + f(listener) + } catch { + case e: Exception => + logError(s"Listener ${Utils.getFormattedClassName(listener)} threw an exception", e) + } + } + } + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala index 5b40a3eb29b30..1ce83485f024b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala @@ -19,10 +19,17 @@ package org.apache.spark.scheduler import collection.mutable.ArrayBuffer +import org.apache.spark.annotation.DeveloperApi + // information about a specific split instance : handles both split instances. // So that we do not need to worry about the differences. -class SplitInfo(val inputFormatClazz: Class[_], val hostLocation: String, val path: String, - val length: Long, val underlyingSplit: Any) { +@DeveloperApi +class SplitInfo( + val inputFormatClazz: Class[_], + val hostLocation: String, + val path: String, + val length: Long, + val underlyingSplit: Any) { override def toString(): String = { "SplitInfo " + super.toString + " .. inputFormatClazz " + inputFormatClazz + ", hostLocation : " + hostLocation + ", path : " + path + 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 5c1fc30e4a557..9a4be43ee219f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.CallSite /** * A stage is a set of independent tasks all computing the same function that need to run as part @@ -35,15 +36,20 @@ import org.apache.spark.storage.BlockManagerId * Each Stage also has a jobId, identifying the job that first submitted the stage. When FIFO * scheduling is used, this allows Stages from earlier jobs to be computed first or recovered * faster on failure. + * + * The callSite provides a location in user code which relates to the stage. For a shuffle map + * stage, the callSite gives the user code that created the RDD being shuffled. For a result + * stage, the callSite gives the user code that executes the associated action (e.g. count()). + * */ private[spark] class Stage( val id: Int, val rdd: RDD[_], val numTasks: Int, - val shuffleDep: Option[ShuffleDependency[_,_]], // Output shuffle if stage is a map stage + val shuffleDep: Option[ShuffleDependency[_, _, _]], // Output shuffle if stage is a map stage val parents: List[Stage], val jobId: Int, - callSite: Option[String]) + val callSite: CallSite) extends Logging { val isShuffleMap = shuffleDep.isDefined @@ -100,7 +106,8 @@ private[spark] class Stage( id } - val name = callSite.getOrElse(rdd.getCreationSite) + val name = callSite.short + val details = callSite.long override def toString = "Stage " + id 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 8115a7ed7896d..7644e3f351b3c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -17,25 +17,46 @@ package org.apache.spark.scheduler +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.storage.RDDInfo /** + * :: DeveloperApi :: * Stores information about a stage to pass from the scheduler to SparkListeners. */ -private[spark] -class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfo: RDDInfo) { +@DeveloperApi +class StageInfo( + val stageId: Int, + val name: String, + val numTasks: Int, + val rddInfos: Seq[RDDInfo], + val details: String) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None + /** Time when all tasks in the stage completed or when the stage was cancelled. */ var completionTime: Option[Long] = None + /** If the stage failed, the reason why. */ + var failureReason: Option[String] = None + var emittedTaskSizeWarning = false + + def stageFailed(reason: String) { + failureReason = Some(reason) + completionTime = Some(System.currentTimeMillis) + } } -private[spark] -object StageInfo { +private[spark] object StageInfo { + /** + * Construct a StageInfo from a Stage. + * + * Each Stage is associated with one or many RDDs, with the boundary of a Stage marked by + * shuffle dependencies. Therefore, all ancestor RDDs related to this Stage's RDD through a + * sequence of narrow dependencies should also be associated with this Stage. + */ def fromStage(stage: Stage): StageInfo = { - val rdd = stage.rdd - val rddName = Option(rdd.name).getOrElse(rdd.id.toString) - val rddInfo = new RDDInfo(rdd.id, rddName, rdd.partitions.size, rdd.getStorageLevel) - new StageInfo(stage.id, stage.name, stage.numTasks, rddInfo) + val ancestorRddInfos = stage.rdd.getNarrowAncestors.map(RDDInfo.fromRdd) + val rddInfos = Seq(RDDInfo.fromRdd(stage.rdd)) ++ ancestorRddInfos + new StageInfo(stage.id, stage.name, stage.numTasks, rddInfos, 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 b85b4a50cd93a..5871edeb856ad 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -17,13 +17,11 @@ package org.apache.spark.scheduler -import java.io.{DataInputStream, DataOutputStream} +import java.io.{ByteArrayOutputStream, DataInputStream, DataOutputStream} import java.nio.ByteBuffer import scala.collection.mutable.HashMap -import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream - import org.apache.spark.TaskContext import org.apache.spark.executor.TaskMetrics import org.apache.spark.serializer.SerializerInstance @@ -35,7 +33,7 @@ import org.apache.spark.util.ByteBufferInputStream * - [[org.apache.spark.scheduler.ResultTask]] * * A Spark job consists of one or more stages. The very last stage in a job consists of multiple - * ResultTask's, while earlier stages consist of ShuffleMapTasks. A ResultTask executes the task + * ResultTasks, while earlier stages consist of ShuffleMapTasks. A ResultTask executes the task * and sends the task output back to the driver application. A ShuffleMapTask executes the task * and divides the task output to multiple buckets (based on the task's partitioner). * @@ -46,8 +44,9 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex final def run(attemptId: Long): T = { context = new TaskContext(stageId, partitionId, attemptId, runningLocally = false) + taskThread = Thread.currentThread() if (_killed) { - kill() + kill(interruptThread = false) } runTask(context) } @@ -64,6 +63,9 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex // Task context, to be initialized in run(). @transient protected var context: TaskContext = _ + // The actual Thread on which the task is running, if any. Initialized in run(). + @volatile @transient private var taskThread: Thread = _ + // A flag to indicate whether the task is killed. This is used in case context is not yet // initialized when kill() is invoked. @volatile @transient private var _killed = false @@ -77,12 +79,16 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex * 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 * be called multiple times. + * If interruptThread is true, we will also call Thread.interrupt() on the Task's executor thread. */ - def kill() { + def kill(interruptThread: Boolean) { _killed = true if (context != null) { context.interrupted = true } + if (interruptThread && taskThread != null) { + taskThread.interrupt() + } } } @@ -104,7 +110,7 @@ private[spark] object Task { serializer: SerializerInstance) : ByteBuffer = { - val out = new FastByteArrayOutputStream(4096) + val out = new ByteArrayOutputStream(4096) val dataOut = new DataOutputStream(out) // Write currentFiles @@ -125,8 +131,7 @@ private[spark] object Task { dataOut.flush() val taskBytes = serializer.serialize(task).array() out.write(taskBytes) - out.trim() - ByteBuffer.wrap(out.array) + ByteBuffer.wrap(out.toByteArray) } /** 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 6183b125def99..4c62e4dc0bac8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -17,10 +17,13 @@ package org.apache.spark.scheduler +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * Information about a running task attempt inside a TaskSet. */ -private[spark] +@DeveloperApi class TaskInfo( val taskId: Long, val index: Int, @@ -46,15 +49,15 @@ class TaskInfo( var serializedSize: Int = 0 - def markGettingResult(time: Long = System.currentTimeMillis) { + private[spark] def markGettingResult(time: Long = System.currentTimeMillis) { gettingResultTime = time } - def markSuccessful(time: Long = System.currentTimeMillis) { + private[spark] def markSuccessful(time: Long = System.currentTimeMillis) { finishTime = time } - def markFailed(time: Long = System.currentTimeMillis) { + private[spark] def markFailed(time: Long = System.currentTimeMillis) { finishTime = time failed = true } @@ -83,11 +86,11 @@ class TaskInfo( def duration: Long = { if (!finished) { - throw new UnsupportedOperationException("duration() called on unfinished tasks") + throw new UnsupportedOperationException("duration() called on unfinished task") } else { finishTime - launchTime } } - def timeRunning(currentTime: Long): Long = currentTime - launchTime + private[spark] def timeRunning(currentTime: Long): Long = currentTime - launchTime } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala index 308edb12edd5c..eb920ab0c0b67 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala @@ -17,7 +17,10 @@ package org.apache.spark.scheduler -private[spark] object TaskLocality extends Enumeration { +import org.apache.spark.annotation.DeveloperApi + +@DeveloperApi +object TaskLocality extends Enumeration { // Process local is expected to be used ONLY within TaskSetManager for now. val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value 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 cb4ad4ae9350c..df59f444b7a0e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -43,7 +43,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul def enqueueSuccessfulTask( taskSetManager: TaskSetManager, tid: Long, serializedData: ByteBuffer) { getTaskResultExecutor.execute(new Runnable { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { try { val result = serializer.get().deserialize[TaskResult[_]](serializedData) match { case directResult: DirectTaskResult[_] => directResult @@ -70,8 +70,9 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul case cnf: ClassNotFoundException => val loader = Thread.currentThread.getContextClassLoader taskSetManager.abort("ClassNotFound with classloader: " + loader) - case ex: Throwable => - taskSetManager.abort("Exception while deserializing and fetching task: %s".format(ex)) + case ex: Exception => + logError("Exception while getting task result", ex) + taskSetManager.abort("Exception while getting task result: %s".format(ex)) } } }) @@ -81,20 +82,20 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul serializedData: ByteBuffer) { var reason : TaskEndReason = UnknownReason getTaskResultExecutor.execute(new Runnable { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { try { if (serializedData != null && serializedData.limit() > 0) { reason = serializer.get().deserialize[TaskEndReason]( - serializedData, getClass.getClassLoader) + serializedData, Utils.getSparkClassLoader) } } catch { case cnd: ClassNotFoundException => // Log an error but keep going here -- the task failed, so not catastropic if we can't // deserialize the reason. - val loader = Thread.currentThread.getContextClassLoader + val loader = Utils.getContextOrSparkClassLoader logError( "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) - case ex: Throwable => {} + case ex: Exception => {} } scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) } 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 92616c997e20c..819c35257b5a7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -47,7 +47,7 @@ private[spark] trait TaskScheduler { def submitTasks(taskSet: TaskSet): Unit // Cancel a stage. - def cancelTasks(stageId: Int) + def cancelTasks(stageId: Int, interruptThread: Boolean) // Set the DAG scheduler for upcalls. This is guaranteed to be set before submitTasks is called. def setDAGScheduler(dagScheduler: DAGScheduler): Unit 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 a92922166f595..17292b4c15b8b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -25,11 +25,13 @@ import scala.concurrent.duration._ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet +import scala.language.postfixOps import scala.util.Random import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.SchedulingMode.SchedulingMode +import org.apache.spark.util.Utils /** * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. @@ -42,7 +44,7 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode * * THREADING: SchedulerBackends and task-submitting clients can call this class from multiple * threads, so it needs locks in public API methods to maintain its state. In addition, some - * SchedulerBackends sycnchronize on themselves when they want to send events here, and then + * SchedulerBackends synchronize on themselves when they want to send events here, and then * acquire a lock on us, so we need to make sure that we don't try to lock the backend while * we are holding a lock on ourselves. */ @@ -98,8 +100,13 @@ private[spark] class TaskSchedulerImpl( var schedulableBuilder: SchedulableBuilder = null var rootPool: Pool = null // default scheduler is FIFO - val schedulingMode: SchedulingMode = SchedulingMode.withName( - conf.get("spark.scheduler.mode", "FIFO")) + private val schedulingModeConf = conf.get("spark.scheduler.mode", "FIFO") + val schedulingMode: SchedulingMode = try { + SchedulingMode.withName(schedulingModeConf.toUpperCase) + } catch { + case e: java.util.NoSuchElementException => + throw new SparkException(s"Unrecognized spark.scheduler.mode: $schedulingModeConf") + } // This is a var so that we can reset it for testing purposes. private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this) @@ -133,7 +140,7 @@ private[spark] class TaskSchedulerImpl( import sc.env.actorSystem.dispatcher sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, SPECULATION_INTERVAL milliseconds) { - checkSpeculatableTasks() + Utils.tryOrExit { checkSpeculatableTasks() } } } } @@ -164,7 +171,7 @@ private[spark] class TaskSchedulerImpl( backend.reviveOffers() } - override def cancelTasks(stageId: Int): Unit = synchronized { + override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = synchronized { logInfo("Cancelling stage " + stageId) activeTaskSets.find(_._2.stageId == stageId).foreach { case (_, tsm) => // There are two possible cases here: @@ -175,7 +182,7 @@ private[spark] class TaskSchedulerImpl( // simply abort the stage. tsm.runningTasksSet.foreach { tid => val execId = taskIdToExecutorId(tid) - backend.killTask(tid, execId) + backend.killTask(tid, execId, interruptThread) } tsm.abort("Stage %s cancelled".format(stageId)) logInfo("Stage %d was cancelled".format(stageId)) @@ -216,7 +223,7 @@ private[spark] class TaskSchedulerImpl( // Build a list of tasks to assign to each worker. val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores)) val availableCpus = shuffledOffers.map(o => o.cores).toArray - val sortedTaskSets = rootPool.getSortedTaskSetQueue() + val sortedTaskSets = rootPool.getSortedTaskSetQueue for (taskSet <- sortedTaskSets) { logDebug("parentName: %s, name: %s, runningTasks: %s".format( taskSet.parent.name, taskSet.name, taskSet.runningTasks)) @@ -350,6 +357,7 @@ private[spark] class TaskSchedulerImpl( if (taskResultGetter != null) { taskResultGetter.stop() } + starvationTimer.cancel() // sleeping for an arbitrary 1 seconds to ensure that messages are sent out. Thread.sleep(1000L) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala index 03bf76083761f..613fa7850bb25 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala @@ -31,8 +31,8 @@ private[spark] class TaskSet( val properties: Properties) { val id: String = stageId + "." + attempt - def kill() { - tasks.foreach(_.kill()) + def kill(interruptThread: Boolean) { + tasks.foreach(_.kill(interruptThread)) } override def toString: String = "TaskSet " + id 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 86d2050a03f18..f3bd0797aa035 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -465,7 +465,7 @@ private[spark] class TaskSetManager( def handleTaskGettingResult(tid: Long) = { val info = taskInfos(tid) info.markGettingResult() - sched.dagScheduler.taskGettingResult(tasks(info.index), info) + sched.dagScheduler.taskGettingResult(info) } /** @@ -538,8 +538,8 @@ private[spark] class TaskSetManager( return } val key = ef.description - failureReason = "Exception failure in TID %s on host %s: %s".format( - tid, info.host, ef.description) + failureReason = "Exception failure in TID %s on host %s: %s\n%s".format( + tid, info.host, ef.description, ef.stackTrace.map(" " + _).mkString("\n")) val now = clock.getTime() val (printFull, dupCount) = { if (recentExceptions.contains(key)) { @@ -582,7 +582,7 @@ private[spark] class TaskSetManager( if (numFailures(index) >= maxTaskFailures) { logError("Task %s:%d failed %d times; aborting job".format( taskSet.id, index, maxTaskFailures)) - abort("Task %s:%d failed %d times (most recent failure: %s)".format( + abort("Task %s:%d failed %d times, most recent failure: %s\nDriver stacktrace:".format( taskSet.id, index, maxTaskFailures, failureReason)) return } 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 4a9a1659d8254..ca74069ef885c 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 @@ -28,9 +28,10 @@ private[spark] sealed trait CoarseGrainedClusterMessage extends Serializable private[spark] object CoarseGrainedClusterMessages { // Driver to executors - case class LaunchTask(task: TaskDescription) extends CoarseGrainedClusterMessage + case class LaunchTask(data: SerializableBuffer) extends CoarseGrainedClusterMessage - case class KillTask(taskId: Long, executor: String) extends CoarseGrainedClusterMessage + case class KillTask(taskId: Long, executor: String, interruptThread: Boolean) + extends CoarseGrainedClusterMessage case class RegisteredExecutor(sparkProperties: Seq[(String, String)]) extends CoarseGrainedClusterMessage 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 7bfc30b4208a3..e47a060683a2d 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 @@ -27,10 +27,10 @@ import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} -import org.apache.spark.{Logging, SparkException, TaskState} +import org.apache.spark.{SparkEnv, Logging, SparkException, TaskState} import org.apache.spark.scheduler.{SchedulerBackend, SlaveLost, TaskDescription, TaskSchedulerImpl, WorkerOffer} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.{SerializableBuffer, AkkaUtils, Utils} /** * A scheduler backend that waits for coarse grained executors to connect to it through Akka. @@ -48,6 +48,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A var totalCoreCount = new AtomicInteger(0) val conf = scheduler.sc.conf private val timeout = AkkaUtils.askTimeout(conf) + private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor { private val executorActor = new HashMap[String, ActorRef] @@ -101,8 +102,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A case ReviveOffers => makeOffers() - case KillTask(taskId, executorId) => - executorActor(executorId) ! KillTask(taskId, executorId) + case KillTask(taskId, executorId, interruptThread) => + executorActor(executorId) ! KillTask(taskId, executorId, interruptThread) case StopDriver => sender ! true @@ -140,8 +141,26 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A // Launch tasks returned by a set of resource offers def launchTasks(tasks: Seq[Seq[TaskDescription]]) { for (task <- tasks.flatten) { - freeCores(task.executorId) -= scheduler.CPUS_PER_TASK - executorActor(task.executorId) ! LaunchTask(task) + val ser = SparkEnv.get.closureSerializer.newInstance() + val serializedTask = ser.serialize(task) + if (serializedTask.limit >= akkaFrameSize - 1024) { + val taskSetId = scheduler.taskIdToTaskSetId(task.taskId) + scheduler.activeTaskSets.get(taskSetId).foreach { taskSet => + try { + var msg = "Serialized task %s:%d was %d bytes which " + + "exceeds spark.akka.frameSize (%d bytes). " + + "Consider using broadcast variables for large values." + msg = msg.format(task.taskId, task.index, serializedTask.limit, akkaFrameSize) + taskSet.abort(msg) + } catch { + case e: Exception => logError("Exception in error callback", e) + } + } + } + else { + freeCores(task.executorId) -= scheduler.CPUS_PER_TASK + executorActor(task.executorId) ! LaunchTask(new SerializableBuffer(serializedTask)) + } } } @@ -207,8 +226,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A driverActor ! ReviveOffers } - override def killTask(taskId: Long, executorId: String) { - driverActor ! KillTask(taskId, executorId) + override def killTask(taskId: Long, executorId: String, interruptThread: Boolean) { + driverActor ! KillTask(taskId, executorId, interruptThread) } override def defaultParallelism(): Int = { 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 25b7472a99cdb..9c07b3f7b695a 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 @@ -42,14 +42,24 @@ private[spark] class SparkDeploySchedulerBackend( // The endpoint for executors to talk to us val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - conf.get("spark.driver.host"), conf.get("spark.driver.port"), + conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") + val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") + val classPathEntries = sc.conf.getOption("spark.executor.extraClassPath").toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } + val libraryPathEntries = + sc.conf.getOption("spark.executor.extraLibraryPath").toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } + val command = Command( - "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) + "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, + classPathEntries, libraryPathEntries, extraJavaOpts) val sparkHome = sc.getSparkHome() val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - sparkHome, sc.ui.appUIAddress, sc.eventLoggingInfo) + sparkHome, sc.ui.appUIAddress, sc.eventLogger.map(_.logDir)) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() @@ -74,10 +84,12 @@ private[spark] class SparkDeploySchedulerBackend( } } - override def dead() { + override def dead(reason: String) { if (!stopping) { - logError("Spark cluster looks dead, giving up.") - scheduler.error("Spark cluster looks down") + logError("Application has been killed. Reason: " + reason) + scheduler.error(reason) + // 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/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 06b041e1fd9a9..9f45400bcf852 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 @@ -111,6 +111,17 @@ private[spark] class CoarseMesosSchedulerBackend( def createCommand(offer: Offer, numCores: Int): CommandInfo = { val environment = Environment.newBuilder() + val extraClassPath = conf.getOption("spark.executor.extraClassPath") + extraClassPath.foreach { cp => + environment.addVariables( + Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build()) + } + val extraJavaOpts = conf.getOption("spark.executor.extraJavaOptions") + + val libraryPathOption = "spark.executor.extraLibraryPath" + val extraLibraryPath = conf.getOption(libraryPathOption).map(p => s"-Djava.library.path=$p") + val extraOpts = Seq(extraJavaOpts, extraLibraryPath).flatten.mkString(" ") + sc.executorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() .setName(key) @@ -123,20 +134,22 @@ private[spark] class CoarseMesosSchedulerBackend( conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) + val uri = conf.get("spark.executor.uri", null) if (uri == null) { val runScript = new File(sparkHome, "./bin/spark-class").getCanonicalPath command.setValue( - "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d".format( - runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) + "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %s %d".format( + runScript, extraOpts, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". val basename = uri.split('/').last.split('.').head command.setValue( ("cd %s*; " + - "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d") - .format(basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) + "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %s %d") + .format(basename, extraOpts, driverUrl, offer.getSlaveId.getValue, + offer.getHostname, numCores)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } command.build() @@ -194,10 +207,12 @@ private[spark] class CoarseMesosSchedulerBackend( .addResources(createResource("cpus", cpusToUse)) .addResources(createResource("mem", sc.executorMemory)) .build() - d.launchTasks(offer.getId, Collections.singletonList(task), filters) + d.launchTasks( + Collections.singleton(offer.getId), Collections.singletonList(task), filters) } else { // Filter it out - d.launchTasks(offer.getId, Collections.emptyList[MesosTaskInfo](), filters) + d.launchTasks( + Collections.singleton(offer.getId), Collections.emptyList[MesosTaskInfo](), filters) } } } 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 dfdcafe19fb93..a089a02d42170 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 @@ -24,7 +24,7 @@ import java.util.Collections import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import com.google.protobuf.ByteString +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, _} @@ -175,7 +175,7 @@ private[spark] class MesosSchedulerBackend( override def reregistered(d: SchedulerDriver, masterInfo: MasterInfo) {} /** - * Method called by Mesos to offer resources on slaves. We resond by asking our active task sets + * Method called by Mesos to offer resources on slaves. We respond by asking our active task sets * for tasks in order of priority. We fill each node with tasks in a round-robin manner so that * tasks are balanced across the cluster. */ @@ -223,7 +223,7 @@ private[spark] class MesosSchedulerBackend( // Reply to the offers val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? for (i <- 0 until offers.size) { - d.launchTasks(offers(i).getId, mesosTasks(i), filters) + d.launchTasks(Collections.singleton(offers(i).getId), mesosTasks(i), filters) } } } finally { 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 16e2f5cf3076d..43f0e18a0cbe0 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 @@ -30,7 +30,7 @@ private case class ReviveOffers() private case class StatusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) -private case class KillTask(taskId: Long) +private case class KillTask(taskId: Long, interruptThread: Boolean) /** * Calls to LocalBackend are all serialized through LocalActor. Using an actor makes the calls on @@ -61,8 +61,8 @@ private[spark] class LocalActor( reviveOffers() } - case KillTask(taskId) => - executor.killTask(taskId) + case KillTask(taskId, interruptThread) => + executor.killTask(taskId, interruptThread) } def reviveOffers() { @@ -99,8 +99,8 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: override def defaultParallelism() = totalCores - override def killTask(taskId: Long, executorId: String) { - localActor ! KillTask(taskId) + override def killTask(taskId: Long, executorId: String, interruptThread: Boolean) { + localActor ! KillTask(taskId, interruptThread) } override def statusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/package-info.java b/core/src/main/scala/org/apache/spark/scheduler/package-info.java new file mode 100644 index 0000000000000..5b4a628d3cee4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/package-info.java @@ -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. + */ + +/** + * Spark's DAG scheduler. + */ +package org.apache.spark.scheduler; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/scheduler/package.scala b/core/src/main/scala/org/apache/spark/scheduler/package.scala new file mode 100644 index 0000000000000..f0dbfc2ac5f48 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/package.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 + +/** + * Spark's scheduling components. This includes the [[org.apache.spark.scheduler.DAGScheduler]] and + * lower level [[org.apache.spark.scheduler.TaskScheduler]]. + */ +package object scheduler 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 18a68b05fa853..0a7e1ec539679 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -20,8 +20,12 @@ package org.apache.spark.serializer import java.io._ import java.nio.ByteBuffer +import scala.reflect.ClassTag + import org.apache.spark.SparkConf +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.ByteBufferInputStream +import org.apache.spark.util.Utils private[spark] class JavaSerializationStream(out: OutputStream, counterReset: Int) extends SerializationStream { @@ -34,7 +38,7 @@ private[spark] class JavaSerializationStream(out: OutputStream, counterReset: In * But only call it every 10,000th time to avoid bloated serialization streams (when * the stream 'resets' object class descriptions have to be re-written) */ - def writeObject[T](t: T): SerializationStream = { + def writeObject[T: ClassTag](t: T): SerializationStream = { objOut.writeObject(t) if (counterReset > 0 && counter >= counterReset) { objOut.reset() @@ -44,6 +48,7 @@ private[spark] class JavaSerializationStream(out: OutputStream, counterReset: In } this } + def flush() { objOut.flush() } def close() { objOut.close() } } @@ -55,12 +60,12 @@ extends DeserializationStream { Class.forName(desc.getName, false, loader) } - def readObject[T](): T = objIn.readObject().asInstanceOf[T] + def readObject[T: ClassTag](): T = objIn.readObject().asInstanceOf[T] def close() { objIn.close() } } private[spark] class JavaSerializerInstance(counterReset: Int) extends SerializerInstance { - def serialize[T](t: T): ByteBuffer = { + def serialize[T: ClassTag](t: T): ByteBuffer = { val bos = new ByteArrayOutputStream() val out = serializeStream(bos) out.writeObject(t) @@ -68,13 +73,13 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize ByteBuffer.wrap(bos.toByteArray) } - def deserialize[T](bytes: ByteBuffer): T = { + def deserialize[T: ClassTag](bytes: ByteBuffer): T = { val bis = new ByteBufferInputStream(bytes) val in = deserializeStream(bis) in.readObject().asInstanceOf[T] } - def deserialize[T](bytes: ByteBuffer, loader: ClassLoader): T = { + def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { val bis = new ByteBufferInputStream(bytes) val in = deserializeStream(bis, loader) in.readObject().asInstanceOf[T] @@ -85,7 +90,7 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize } def deserializeStream(s: InputStream): DeserializationStream = { - new JavaDeserializationStream(s, Thread.currentThread.getContextClassLoader) + new JavaDeserializationStream(s, Utils.getContextOrSparkClassLoader) } def deserializeStream(s: InputStream, loader: ClassLoader): DeserializationStream = { @@ -94,8 +99,14 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize } /** + * :: DeveloperApi :: * A Spark serializer that uses Java's built-in serialization. + * + * Note that this serializer is not guaranteed to be wire-compatible across different versions of + * Spark. It is intended to be used to serialize/de-serialize data within a single + * Spark application. */ +@DeveloperApi class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable { private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 10000) 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 5352f03701f75..cf0c495c6c2c0 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -31,8 +31,14 @@ import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage._ import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock} +import scala.reflect.ClassTag + /** * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]]. + * + * Note that this serializer is not guaranteed to be wire-compatible across different versions of + * Spark. It is intended to be used to serialize/de-serialize data within a single + * Spark application. */ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serializer @@ -55,7 +61,9 @@ class KryoSerializer(conf: SparkConf) // Do this before we invoke the user registrator so the user registrator can override this. kryo.setReferences(referenceTracking) - for (cls <- KryoSerializer.toRegister) kryo.register(cls) + for (cls <- KryoSerializer.toRegister) { + kryo.register(cls) + } // Allow sending SerializableWritable kryo.register(classOf[SerializableWritable[_]], new KryoJavaSerializer()) @@ -74,7 +82,7 @@ class KryoSerializer(conf: SparkConf) } // Register Chill's classes; we do this after our ranges and the user's own classes to let - // our code override the generic serialziers in Chill for things like Seq + // our code override the generic serializers in Chill for things like Seq new AllScalaRegistrar().apply(kryo) kryo.setClassLoader(classLoader) @@ -90,7 +98,7 @@ private[spark] class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { val output = new KryoOutput(outStream) - def writeObject[T](t: T): SerializationStream = { + def writeObject[T: ClassTag](t: T): SerializationStream = { kryo.writeClassAndObject(output, t) this } @@ -103,7 +111,7 @@ private[spark] class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream { val input = new KryoInput(inStream) - def readObject[T](): T = { + def readObject[T: ClassTag](): T = { try { kryo.readClassAndObject(input).asInstanceOf[T] } catch { @@ -126,18 +134,18 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ lazy val output = ks.newKryoOutput() lazy val input = new KryoInput() - def serialize[T](t: T): ByteBuffer = { + def serialize[T: ClassTag](t: T): ByteBuffer = { output.clear() kryo.writeClassAndObject(output, t) ByteBuffer.wrap(output.toBytes) } - def deserialize[T](bytes: ByteBuffer): T = { + def deserialize[T: ClassTag](bytes: ByteBuffer): T = { input.setBuffer(bytes.array) kryo.readClassAndObject(input).asInstanceOf[T] } - def deserialize[T](bytes: ByteBuffer, loader: ClassLoader): T = { + def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { val oldClassLoader = kryo.getClassLoader kryo.setClassLoader(loader) input.setBuffer(bytes.array) @@ -173,10 +181,6 @@ private[serializer] object KryoSerializer { classOf[GetBlock], classOf[MapStatus], classOf[BlockManagerId], - classOf[Array[Byte]], - (1 to 10).getClass, - (1 until 10).getClass, - (1L to 10L).getClass, - (1L until 10L).getClass + classOf[Array[Byte]] ) } 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 099143494b851..f2f5cea469c61 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -17,25 +17,32 @@ package org.apache.spark.serializer -import java.io.{EOFException, InputStream, OutputStream} +import java.io.{ByteArrayOutputStream, EOFException, InputStream, OutputStream} import java.nio.ByteBuffer -import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream +import scala.reflect.ClassTag -import org.apache.spark.util.{ByteBufferInputStream, NextIterator} import org.apache.spark.SparkEnv +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.{ByteBufferInputStream, NextIterator} /** + * :: DeveloperApi :: * A serializer. Because some serialization libraries are not thread safe, this class is used to * create [[org.apache.spark.serializer.SerializerInstance]] objects that do the actual * serialization and are guaranteed to only be called from one thread at a time. * * Implementations of this trait should implement: + * * 1. a zero-arg constructor or a constructor that accepts a [[org.apache.spark.SparkConf]] * as parameter. If both constructors are defined, the latter takes precedence. * * 2. Java serialization interface. + * + * Note that serializers are not required to be wire-compatible across different versions of Spark. + * They are intended to be used to serialize/de-serialize data within a single Spark application. */ +@DeveloperApi trait Serializer { def newInstance(): SerializerInstance } @@ -45,29 +52,34 @@ object Serializer { def getSerializer(serializer: Serializer): Serializer = { if (serializer == null) SparkEnv.get.serializer else serializer } + + def getSerializer(serializer: Option[Serializer]): Serializer = { + serializer.getOrElse(SparkEnv.get.serializer) + } } /** + * :: DeveloperApi :: * An instance of a serializer, for use by one thread at a time. */ +@DeveloperApi trait SerializerInstance { - def serialize[T](t: T): ByteBuffer + def serialize[T: ClassTag](t: T): ByteBuffer - def deserialize[T](bytes: ByteBuffer): T + def deserialize[T: ClassTag](bytes: ByteBuffer): T - def deserialize[T](bytes: ByteBuffer, loader: ClassLoader): T + def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T def serializeStream(s: OutputStream): SerializationStream def deserializeStream(s: InputStream): DeserializationStream - def serializeMany[T](iterator: Iterator[T]): ByteBuffer = { + def serializeMany[T: ClassTag](iterator: Iterator[T]): ByteBuffer = { // Default implementation uses serializeStream - val stream = new FastByteArrayOutputStream() + val stream = new ByteArrayOutputStream() serializeStream(stream).writeAll(iterator) - val buffer = ByteBuffer.allocate(stream.position.toInt) - buffer.put(stream.array, 0, stream.position.toInt) + val buffer = ByteBuffer.wrap(stream.toByteArray) buffer.flip() buffer } @@ -79,16 +91,17 @@ trait SerializerInstance { } } - /** + * :: DeveloperApi :: * A stream for writing serialized objects. */ +@DeveloperApi trait SerializationStream { - def writeObject[T](t: T): SerializationStream + def writeObject[T: ClassTag](t: T): SerializationStream def flush(): Unit def close(): Unit - def writeAll[T](iter: Iterator[T]): SerializationStream = { + def writeAll[T: ClassTag](iter: Iterator[T]): SerializationStream = { while (iter.hasNext) { writeObject(iter.next()) } @@ -98,10 +111,12 @@ trait SerializationStream { /** + * :: DeveloperApi :: * A stream for reading serialized objects. */ +@DeveloperApi trait DeserializationStream { - def readObject[T](): T + def readObject[T: ClassTag](): T def close(): Unit /** diff --git a/core/src/main/scala/org/apache/spark/serializer/package-info.java b/core/src/main/scala/org/apache/spark/serializer/package-info.java new file mode 100644 index 0000000000000..4c0b73ab36a00 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/serializer/package-info.java @@ -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. + */ + +/** + * Pluggable serializers for RDD and shuffle data. + */ +package org.apache.spark.serializer; \ No newline at end of file diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SparseMatrix.scala b/core/src/main/scala/org/apache/spark/serializer/package.scala similarity index 74% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/SparseMatrix.scala rename to core/src/main/scala/org/apache/spark/serializer/package.scala index cbd1a2a5a4bd8..8f09ef6f5b34c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SparseMatrix.scala +++ b/core/src/main/scala/org/apache/spark/serializer/package.scala @@ -15,16 +15,11 @@ * limitations under the License. */ -package org.apache.spark.mllib.linalg - -import org.apache.spark.rdd.RDD - +package org.apache.spark /** - * Class that represents a sparse matrix + * Pluggable serializers for RDD and shuffle data. * - * @param data RDD of nonzero entries - * @param m number of rows - * @param n numner of columns + * @see [[org.apache.spark.serializer.Serializer]] */ -case class SparseMatrix(val data: RDD[MatrixEntry], val m: Int, val n: Int) +package object serializer diff --git a/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/BaseShuffleHandle.scala similarity index 66% rename from core/src/main/scala/org/apache/spark/ShuffleFetcher.scala rename to core/src/main/scala/org/apache/spark/shuffle/BaseShuffleHandle.scala index a4f69b6b22b2c..b36c457d6d514 100644 --- a/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BaseShuffleHandle.scala @@ -15,22 +15,16 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.shuffle +import org.apache.spark.{ShuffleDependency, Aggregator, Partitioner} import org.apache.spark.serializer.Serializer -private[spark] abstract class ShuffleFetcher { - - /** - * Fetch the shuffle outputs for a given ShuffleDependency. - * @return An iterator over the elements of the fetched shuffle outputs. - */ - def fetch[T]( - shuffleId: Int, - reduceId: Int, - context: TaskContext, - serializer: Serializer = SparkEnv.get.serializer): Iterator[T] - - /** Stop the fetcher */ - def stop() {} -} +/** + * A basic ShuffleHandle implementation that just captures registerShuffle's parameters. + */ +private[spark] class BaseShuffleHandle[K, V, C]( + shuffleId: Int, + val numMaps: Int, + val dependency: ShuffleDependency[K, V, C]) + extends ShuffleHandle(shuffleId) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleHandle.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleHandle.scala new file mode 100644 index 0000000000000..13c7115f88afa --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleHandle.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle + +/** + * 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 {} diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala new file mode 100644 index 0000000000000..9c859b8b4a118 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.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.shuffle + +import org.apache.spark.{TaskContext, ShuffleDependency} + +/** + * Pluggable interface for shuffle systems. A ShuffleManager is created in SparkEnv on both the + * driver and executors, based on the spark.shuffle.manager setting. The driver registers shuffles + * with it, and executors (or tasks running locally in the driver) can ask to read and write data. + * + * NOTE: this will be instantiated by SparkEnv so its constructor can take a SparkConf and + * boolean isDriver as parameters. + */ +private[spark] trait ShuffleManager { + /** + * Register a shuffle with the manager and obtain a handle for it to pass to tasks. + */ + def registerShuffle[K, V, C]( + shuffleId: Int, + numMaps: Int, + dependency: ShuffleDependency[K, V, C]): ShuffleHandle + + /** Get a writer for a given partition. Called on executors by map tasks. */ + def getWriter[K, V](handle: ShuffleHandle, mapId: Int, context: TaskContext): ShuffleWriter[K, V] + + /** + * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive). + * Called on executors by reduce tasks. + */ + def getReader[K, C]( + handle: ShuffleHandle, + startPartition: Int, + endPartition: Int, + context: TaskContext): ShuffleReader[K, C] + + /** Remove a shuffle's metadata from the ShuffleManager. */ + def unregisterShuffle(shuffleId: Int) + + /** Shut down this ShuffleManager. */ + def stop(): Unit +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleReader.scala new file mode 100644 index 0000000000000..b30e366d06006 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleReader.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.shuffle + +/** + * Obtained inside a reduce task to read combined records from the mappers. + */ +private[spark] trait ShuffleReader[K, C] { + /** Read the combined key-values for this reduce task */ + def read(): Iterator[Product2[K, C]] + + /** Close this reader */ + 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 new file mode 100644 index 0000000000000..ead3ebd652ca5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.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.shuffle + +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 record to this task's output */ + def write(record: Product2[K, V]): Unit + + /** Close this writer, passing along whether the map completed */ + def stop(success: Boolean): Option[MapStatus] +} diff --git a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala rename to core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala index a67392441ed29..b05b6ea345df3 100644 --- a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.shuffle.hash import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap @@ -24,17 +24,16 @@ import org.apache.spark.executor.ShuffleReadMetrics import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockId} import org.apache.spark.util.CompletionIterator +import org.apache.spark._ -private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { - - override def fetch[T]( +private[hash] object BlockStoreShuffleFetcher extends Logging { + def fetch[T]( shuffleId: Int, reduceId: Int, context: TaskContext, serializer: Serializer) : Iterator[T] = { - logDebug("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId)) val blockManager = SparkEnv.get.blockManager 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 new file mode 100644 index 0000000000000..5b0940ecce29d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.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.shuffle.hash + +import org.apache.spark._ +import org.apache.spark.shuffle._ + +/** + * A ShuffleManager using hashing, that creates one output file per reduce partition on each + * mapper (possibly reusing these across waves of tasks). + */ +class HashShuffleManager(conf: SparkConf) extends ShuffleManager { + /* Register a shuffle with the manager and obtain a handle for it to pass to tasks. */ + override def registerShuffle[K, V, C]( + shuffleId: Int, + numMaps: Int, + dependency: ShuffleDependency[K, V, C]): ShuffleHandle = { + new BaseShuffleHandle(shuffleId, numMaps, dependency) + } + + /** + * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive). + * Called on executors by reduce tasks. + */ + override def getReader[K, C]( + handle: ShuffleHandle, + startPartition: Int, + endPartition: Int, + context: TaskContext): ShuffleReader[K, C] = { + new HashShuffleReader( + handle.asInstanceOf[BaseShuffleHandle[K, _, C]], startPartition, endPartition, context) + } + + /** Get a writer for a given partition. Called on executors by map tasks. */ + override def getWriter[K, V](handle: ShuffleHandle, mapId: Int, context: TaskContext) + : ShuffleWriter[K, V] = { + new HashShuffleWriter(handle.asInstanceOf[BaseShuffleHandle[K, V, _]], mapId, context) + } + + /** Remove a shuffle's metadata from the ShuffleManager. */ + override def unregisterShuffle(shuffleId: Int): Unit = {} + + /** Shut down this ShuffleManager. */ + override def stop(): Unit = {} +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala new file mode 100644 index 0000000000000..f6a790309a587 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala @@ -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. + */ + +package org.apache.spark.shuffle.hash + +import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.{BaseShuffleHandle, ShuffleReader} +import org.apache.spark.TaskContext + +class HashShuffleReader[K, C]( + handle: BaseShuffleHandle[K, _, C], + startPartition: Int, + endPartition: Int, + context: TaskContext) + extends ShuffleReader[K, C] +{ + require(endPartition == startPartition + 1, + "Hash shuffle currently only supports fetching one partition") + + /** Read the combined key-values for this reduce task */ + override def read(): Iterator[Product2[K, C]] = { + BlockStoreShuffleFetcher.fetch(handle.shuffleId, startPartition, context, + Serializer.getSerializer(handle.dependency.serializer)) + } + + /** Close this reader */ + override def stop(): Unit = ??? +} 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 new file mode 100644 index 0000000000000..4c6749098c110 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle.hash + +import org.apache.spark.shuffle.{BaseShuffleHandle, ShuffleWriter} +import org.apache.spark.{Logging, MapOutputTracker, SparkEnv, TaskContext} +import org.apache.spark.storage.{BlockObjectWriter} +import org.apache.spark.serializer.Serializer +import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.scheduler.MapStatus + +class HashShuffleWriter[K, V]( + handle: BaseShuffleHandle[K, V, _], + mapId: Int, + context: TaskContext) + extends ShuffleWriter[K, V] with Logging { + + private val dep = handle.dependency + private val numOutputSplits = dep.partitioner.numPartitions + private val metrics = context.taskMetrics + private var stopping = false + + private val blockManager = SparkEnv.get.blockManager + private val shuffleBlockManager = blockManager.shuffleBlockManager + private val ser = Serializer.getSerializer(dep.serializer.getOrElse(null)) + private val shuffle = shuffleBlockManager.forMapTask(dep.shuffleId, mapId, numOutputSplits, ser) + + /** Write a record to this task's output */ + override def write(record: Product2[K, V]): Unit = { + val pair = record.asInstanceOf[Product2[Any, Any]] + val bucketId = dep.partitioner.getPartition(pair._1) + shuffle.writers(bucketId).write(pair) + } + + /** Close this writer, passing along whether the map completed */ + override def stop(success: Boolean): Option[MapStatus] = { + try { + if (stopping) { + return None + } + stopping = true + if (success) { + try { + return Some(commitWritesAndBuildStatus()) + } catch { + case e: Exception => + revertWrites() + throw e + } + } else { + revertWrites() + return None + } + } finally { + // Release the writers back to the shuffle block manager. + if (shuffle != null && shuffle.writers != null) { + try { + shuffle.releaseWriters(success) + } catch { + case e: Exception => logError("Failed to release shuffle writers", e) + } + } + } + } + + private def commitWritesAndBuildStatus(): MapStatus = { + // Commit the writes. Get the size of each bucket block (total block size). + var totalBytes = 0L + var totalTime = 0L + val compressedSizes = shuffle.writers.map { writer: BlockObjectWriter => + writer.commit() + writer.close() + val size = writer.fileSegment().length + totalBytes += size + totalTime += writer.timeWriting() + MapOutputTracker.compressSize(size) + } + + // Update shuffle metrics. + val shuffleMetrics = new ShuffleWriteMetrics + shuffleMetrics.shuffleBytesWritten = totalBytes + shuffleMetrics.shuffleWriteTime = totalTime + metrics.shuffleWriteMetrics = Some(shuffleMetrics) + + new MapStatus(blockManager.blockManagerId, compressedSizes) + } + + private def revertWrites(): Unit = { + if (shuffle != null && shuffle.writers != null) { + for (writer <- shuffle.writers) { + writer.revertPartialWrites() + writer.close() + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 2fbbda5b76c74..408a797088059 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -118,11 +118,9 @@ object BlockFetcherIterator { }) bytesInFlight += req.size val sizeMap = req.blocks.toMap // so we can look up the size of each blockID - val fetchStart = System.currentTimeMillis() val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) future.onSuccess { case Some(message) => { - val fetchDone = System.currentTimeMillis() val bufferMessage = message.asInstanceOf[BufferMessage] val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) for (blockMessage <- blockMessageArray) { @@ -148,6 +146,12 @@ object BlockFetcherIterator { } protected def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { + // Make remote requests at most maxBytesInFlight / 5 in length; the reason to keep them + // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 + // nodes, rather than blocking on reading output from one node. + val targetRequestSize = math.max(maxBytesInFlight / 5, 1L) + logInfo("maxBytesInFlight: " + maxBytesInFlight + ", targetRequestSize: " + targetRequestSize) + // Split local and remote blocks. Remote blocks are further split into FetchRequests of size // at most maxBytesInFlight in order to limit the amount of data in flight. val remoteRequests = new ArrayBuffer[FetchRequest] @@ -159,11 +163,6 @@ object BlockFetcherIterator { _numBlocksToFetch += localBlocksToFetch.size } else { numRemote += blockInfos.size - // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them - // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 - // nodes, rather than blocking on reading output from one node. - val minRequestSize = math.max(maxBytesInFlight / 5, 1L) - logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) val iterator = blockInfos.iterator var curRequestSize = 0L var curBlocks = new ArrayBuffer[(BlockId, Long)] @@ -178,11 +177,12 @@ object BlockFetcherIterator { } else if (size < 0) { throw new BlockException(blockId, "Negative block size " + size) } - if (curRequestSize >= minRequestSize) { + if (curRequestSize >= targetRequestSize) { // Add this FetchRequest remoteRequests += new FetchRequest(address, curBlocks) curRequestSize = 0 curBlocks = new ArrayBuffer[(BlockId, Long)] + logDebug(s"Creating fetch request of $curRequestSize at $address") } } // Add in the final request @@ -191,7 +191,7 @@ object BlockFetcherIterator { } } } - logInfo("Getting " + _numBlocksToFetch + " non-zero-bytes blocks out of " + + logInfo("Getting " + _numBlocksToFetch + " non-empty blocks out of " + totalBlocks + " blocks") remoteRequests } @@ -226,8 +226,8 @@ object BlockFetcherIterator { sendRequest(fetchRequests.dequeue()) } - val numGets = remoteRequests.size - fetchRequests.size - logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime)) + val numFetches = remoteRequests.size - fetchRequests.size + logInfo("Started " + numFetches + " remote fetches in" + Utils.getUsedTimeMs(startTime)) // Get Local Blocks startTime = System.currentTimeMillis @@ -240,7 +240,7 @@ object BlockFetcherIterator { override def numRemoteBlocks: Int = numRemote override def fetchWaitTime: Long = _fetchWaitTime override def remoteBytesRead: Long = _remoteBytesRead - + // Implementing the Iterator methods with an iterator that reads fetched blocks off the queue // as they arrive. @@ -327,7 +327,7 @@ object BlockFetcherIterator { } copiers = startCopiers(conf.getInt("spark.shuffle.copier.threads", 6)) - logInfo("Started " + fetchRequestsSync.size + " remote gets in " + + logInfo("Started " + fetchRequestsSync.size + " remote fetches in " + Utils.getUsedTimeMs(startTime)) // Get Local Blocks 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 301d784b350a3..42ec181b00bb3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -19,14 +19,18 @@ package org.apache.spark.storage import java.util.UUID +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * Identifies a particular Block of data, usually associated with a single file. * A Block can be uniquely identified by its filename, but each type of Block has a different * set of keys which produce its unique name. * * If your BlockId should be serializable, be sure to add it to the BlockId.apply() method. */ -private[spark] sealed abstract class BlockId { +@DeveloperApi +sealed abstract class BlockId { /** A globally unique identifier for this Block. Can be used for ser/de. */ def name: String @@ -34,7 +38,7 @@ private[spark] sealed abstract class BlockId { def asRDDId = if (isRDD) Some(asInstanceOf[RDDBlockId]) else None def isRDD = isInstanceOf[RDDBlockId] def isShuffle = isInstanceOf[ShuffleBlockId] - def isBroadcast = isInstanceOf[BroadcastBlockId] || isInstanceOf[BroadcastHelperBlockId] + def isBroadcast = isInstanceOf[BroadcastBlockId] override def toString = name override def hashCode = name.hashCode @@ -44,29 +48,29 @@ private[spark] sealed abstract class BlockId { } } -private[spark] case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { +@DeveloperApi +case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { def name = "rdd_" + rddId + "_" + splitIndex } -private[spark] -case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { +@DeveloperApi +case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) + extends BlockId { def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId } -private[spark] case class BroadcastBlockId(broadcastId: Long) extends BlockId { - def name = "broadcast_" + broadcastId -} - -private[spark] -case class BroadcastHelperBlockId(broadcastId: BroadcastBlockId, hType: String) extends BlockId { - def name = broadcastId.name + "_" + hType +@DeveloperApi +case class BroadcastBlockId(broadcastId: Long, field: String = "") extends BlockId { + def name = "broadcast_" + broadcastId + (if (field == "") "" else "_" + field) } -private[spark] case class TaskResultBlockId(taskId: Long) extends BlockId { +@DeveloperApi +case class TaskResultBlockId(taskId: Long) extends BlockId { def name = "taskresult_" + taskId } -private[spark] case class StreamBlockId(streamId: Int, uniqueId: Long) extends BlockId { +@DeveloperApi +case class StreamBlockId(streamId: Int, uniqueId: Long) extends BlockId { def name = "input-" + streamId + "-" + uniqueId } @@ -80,11 +84,11 @@ private[spark] case class TestBlockId(id: String) extends BlockId { def name = "test_" + id } -private[spark] object BlockId { +@DeveloperApi +object BlockId { val RDD = "rdd_([0-9]+)_([0-9]+)".r val SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)".r - val BROADCAST = "broadcast_([0-9]+)".r - val BROADCAST_HELPER = "broadcast_([0-9]+)_([A-Za-z0-9]+)".r + val BROADCAST = "broadcast_([0-9]+)([_A-Za-z0-9]*)".r val TASKRESULT = "taskresult_([0-9]+)".r val STREAM = "input-([0-9]+)-([0-9]+)".r val TEST = "test_(.*)".r @@ -95,10 +99,8 @@ private[spark] object BlockId { RDDBlockId(rddId.toInt, splitIndex.toInt) case SHUFFLE(shuffleId, mapId, reduceId) => ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) - case BROADCAST(broadcastId) => - BroadcastBlockId(broadcastId.toLong) - case BROADCAST_HELPER(broadcastId, hType) => - BroadcastHelperBlockId(BroadcastBlockId(broadcastId.toLong), hType) + case BROADCAST(broadcastId, field) => + BroadcastBlockId(broadcastId.toLong, field.stripPrefix("_")) case TASKRESULT(taskId) => TaskResultBlockId(taskId.toLong) case STREAM(streamId, uniqueId) => diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala index c8f397609a0b4..22fdf73e9d1f4 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala @@ -29,9 +29,9 @@ private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolea setInitThread() private def setInitThread() { - // Set current thread as init thread - waitForReady will not block this thread - // (in case there is non trivial initialization which ends up calling waitForReady as part of - // initialization itself) + /* Set current thread as init thread - waitForReady will not block this thread + * (in case there is non trivial initialization which ends up calling waitForReady + * as part of initialization itself) */ BlockInfo.blockInfoInitThreads.put(this, Thread.currentThread()) } @@ -42,7 +42,9 @@ private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolea def waitForReady(): Boolean = { if (pending && initThread != Thread.currentThread()) { synchronized { - while (pending) this.wait() + while (pending) { + this.wait() + } } } !failed @@ -50,8 +52,8 @@ private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolea /** Mark this BlockInfo as ready (i.e. block is finished writing) */ def markReady(sizeInBytes: Long) { - require (sizeInBytes >= 0, "sizeInBytes was negative: " + sizeInBytes) - assert (pending) + require(sizeInBytes >= 0, s"sizeInBytes was negative: $sizeInBytes") + assert(pending) size = sizeInBytes BlockInfo.blockInfoInitThreads.remove(this) synchronized { @@ -61,7 +63,7 @@ private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolea /** Mark this BlockInfo as ready but failed */ def markFailure() { - assert (pending) + assert(pending) size = BlockInfo.BLOCK_FAILED BlockInfo.blockInfoInitThreads.remove(this) synchronized { @@ -71,9 +73,9 @@ private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolea } private object BlockInfo { - // initThread is logically a BlockInfo field, but we store it here because - // it's only needed while this block is in the 'pending' state and we want - // to minimize BlockInfo's memory footprint. + /* initThread is logically a BlockInfo field, but we store it here because + * it's only needed while this block is in the 'pending' state and we want + * to minimize BlockInfo's memory footprint. */ private val blockInfoInitThreads = new ConcurrentHashMap[BlockInfo, Thread] private val BLOCK_PENDING: Long = -1L 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 19138d9dde697..d2f7baf928b62 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -17,54 +17,59 @@ package org.apache.spark.storage -import java.io.{File, InputStream, OutputStream} +import java.io.{File, InputStream, OutputStream, BufferedOutputStream, ByteArrayOutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} + import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.{Await, Future} import scala.concurrent.duration._ import scala.util.Random + import akka.actor.{ActorSystem, Cancellable, Props} -import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream} import sun.nio.ch.DirectBuffer -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv, SparkException} + +import org.apache.spark._ import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer import org.apache.spark.util._ - -sealed trait Values - -case class ByteBufferValues(buffer: ByteBuffer) extends Values -case class IteratorValues(iterator: Iterator[Any]) extends Values -case class ArrayBufferValues(buffer: ArrayBuffer[Any]) extends Values +private[spark] sealed trait BlockValues +private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends BlockValues +private[spark] case class IteratorValues(iterator: Iterator[Any]) extends BlockValues +private[spark] case class ArrayBufferValues(buffer: ArrayBuffer[Any]) extends BlockValues private[spark] class BlockManager( executorId: String, actorSystem: ActorSystem, val master: BlockManagerMaster, - val defaultSerializer: Serializer, + defaultSerializer: Serializer, maxMemory: Long, val conf: SparkConf, - securityManager: SecurityManager) + securityManager: SecurityManager, + mapOutputTracker: MapOutputTracker) extends Logging { val shuffleBlockManager = new ShuffleBlockManager(this) val diskBlockManager = new DiskBlockManager(shuffleBlockManager, - conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) + conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) + val connectionManager = new ConnectionManager(0, conf, securityManager) + + implicit val futureExecContext = connectionManager.futureExecContext private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo] - private[storage] val memoryStore: BlockStore = new MemoryStore(this, maxMemory) + // Actual storage of where blocks are kept + private var tachyonInitialized = false + private[storage] val memoryStore = new MemoryStore(this, maxMemory) private[storage] val diskStore = new DiskStore(this, diskBlockManager) - var tachyonInitialized = false private[storage] 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 tachyonStorePath = s"$storeDir/$appFolderName/${this.executorId}" val tachyonMaster = conf.get("spark.tachyonStore.url", "tachyon://localhost:19998") - val tachyonBlockManager = new TachyonBlockManager( - shuffleBlockManager, tachyonStorePath, tachyonMaster) + val tachyonBlockManager = + new TachyonBlockManager(shuffleBlockManager, tachyonStorePath, tachyonMaster) tachyonInitialized = true new TachyonStore(this, tachyonBlockManager) } @@ -76,43 +81,39 @@ private[spark] class BlockManager( if (useNetty) diskBlockManager.startShuffleBlockSender(nettyPortConfig) else 0 } - val connectionManager = new ConnectionManager(0, conf, securityManager) - implicit val futureExecContext = connectionManager.futureExecContext - val blockManagerId = BlockManagerId( executorId, connectionManager.id.host, connectionManager.id.port, nettyPort) // Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory // for receiving shuffle outputs) - val maxBytesInFlight = - conf.getLong("spark.reducer.maxMbInFlight", 48) * 1024 * 1024 + val maxBytesInFlight = conf.getLong("spark.reducer.maxMbInFlight", 48) * 1024 * 1024 // Whether to compress broadcast variables that are stored - val compressBroadcast = conf.getBoolean("spark.broadcast.compress", true) + private val compressBroadcast = conf.getBoolean("spark.broadcast.compress", true) // Whether to compress shuffle output that are stored - val compressShuffle = conf.getBoolean("spark.shuffle.compress", true) + private val compressShuffle = conf.getBoolean("spark.shuffle.compress", true) // Whether to compress RDD partitions that are stored serialized - val compressRdds = conf.getBoolean("spark.rdd.compress", false) + private val compressRdds = conf.getBoolean("spark.rdd.compress", false) // Whether to compress shuffle output temporarily spilled to disk - val compressShuffleSpill = conf.getBoolean("spark.shuffle.spill.compress", true) - - val heartBeatFrequency = BlockManager.getHeartBeatFrequency(conf) + private val compressShuffleSpill = conf.getBoolean("spark.shuffle.spill.compress", true) - val slaveActor = actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)), + private val slaveActor = actorSystem.actorOf( + Props(new BlockManagerSlaveActor(this, mapOutputTracker)), name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next) - // Pending re-registration action being executed asynchronously or null if none - // is pending. Accesses should synchronize on asyncReregisterLock. - var asyncReregisterTask: Future[Unit] = null - val asyncReregisterLock = new Object + // Pending re-registration action being executed asynchronously or null if none is pending. + // Accesses should synchronize on asyncReregisterLock. + private var asyncReregisterTask: Future[Unit] = null + private val asyncReregisterLock = new Object - private def heartBeat() { + private def heartBeat(): Unit = { if (!master.sendHeartBeat(blockManagerId)) { reregister() } } - var heartBeatTask: Cancellable = null + private val heartBeatFrequency = BlockManager.getHeartBeatFrequency(conf) + private var heartBeatTask: Cancellable = null private val metadataCleaner = new MetadataCleaner( MetadataCleanerType.BLOCK_MANAGER, this.dropOldNonBroadcastBlocks, conf) @@ -121,11 +122,11 @@ private[spark] class BlockManager( initialize() - // The compression codec to use. Note that the "lazy" val is necessary because we want to delay - // the initialization of the compression codec until it is first used. The reason is that a Spark - // program could be using a user-defined codec in a third party jar, which is loaded in - // Executor.updateDependencies. When the BlockManager is initialized, user level jars hasn't been - // loaded yet. + /* The compression codec to use. Note that the "lazy" val is necessary because we want to delay + * the initialization of the compression codec until it is first used. The reason is that a Spark + * program could be using a user-defined codec in a third party jar, which is loaded in + * Executor.updateDependencies. When the BlockManager is initialized, user level jars hasn't been + * loaded yet. */ private lazy val compressionCodec: CompressionCodec = CompressionCodec.createCodec(conf) /** @@ -137,21 +138,22 @@ private[spark] class BlockManager( master: BlockManagerMaster, serializer: Serializer, conf: SparkConf, - securityManager: SecurityManager) = { + securityManager: SecurityManager, + mapOutputTracker: MapOutputTracker) = { this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), - conf, securityManager) + conf, securityManager, mapOutputTracker) } /** * Initialize the BlockManager. Register to the BlockManagerMaster, and start the * BlockManagerWorker actor. */ - private def initialize() { + private def initialize(): Unit = { master.registerBlockManager(blockManagerId, maxMemory, slaveActor) BlockManagerWorker.startBlockManagerWorker(this) if (!BlockManager.getDisableHeartBeatsForTesting(conf)) { heartBeatTask = actorSystem.scheduler.schedule(0.seconds, heartBeatFrequency.milliseconds) { - heartBeat() + Utils.tryOrExit { heartBeat() } } } } @@ -166,12 +168,12 @@ private[spark] class BlockManager( * heart beat attempt or new block registration and another try to re-register all blocks * will be made then. */ - private def reportAllBlocks() { - logInfo("Reporting " + blockInfo.size + " blocks to the master.") + private def reportAllBlocks(): Unit = { + logInfo(s"Reporting ${blockInfo.size} blocks to the master.") for ((blockId, info) <- blockInfo) { val status = getCurrentBlockStatus(blockId, info) if (!tryToReportBlockStatus(blockId, info, status)) { - logError("Failed to report " + blockId + " to master; giving up.") + logError(s"Failed to report $blockId to master; giving up.") return } } @@ -183,7 +185,7 @@ private[spark] class BlockManager( * * Note that this method must be called without any BlockInfo locks held. */ - def reregister() { + private def reregister(): Unit = { // TODO: We might need to rate limit re-registering. logInfo("BlockManager re-registering with master") master.registerBlockManager(blockManagerId, maxMemory, slaveActor) @@ -193,7 +195,7 @@ private[spark] class BlockManager( /** * Re-register with the master sometime soon. */ - def asyncReregister() { + private def asyncReregister(): Unit = { asyncReregisterLock.synchronized { if (asyncReregisterTask == null) { asyncReregisterTask = Future[Unit] { @@ -209,7 +211,7 @@ private[spark] class BlockManager( /** * For testing. Wait for any pending asynchronous re-registration; otherwise, do nothing. */ - def waitForAsyncReregister() { + def waitForAsyncReregister(): Unit = { val task = asyncReregisterTask if (task != null) { Await.ready(task, Duration.Inf) @@ -217,9 +219,26 @@ private[spark] class BlockManager( } /** - * Get storage level of local block. If no info exists for the block, then returns null. + * 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. + */ + 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 + BlockStatus(info.level, memSize, diskSize, 0L) + } + } + + /** + * Get the ids of existing blocks that match the given filter. Note that this will + * query the blocks stored in the disk block manager (that the block manager + * may not know of). */ - def getLevel(blockId: BlockId): StorageLevel = blockInfo.get(blockId).map(_.level).orNull + def getMatchingBlockIds(filter: BlockId => Boolean): Seq[BlockId] = { + (blockInfo.keys ++ diskBlockManager.getAllBlocks()).filter(filter).toSeq + } /** * Tell the master about the current storage status of a block. This will send a block update @@ -230,18 +249,18 @@ private[spark] class BlockManager( * it is still valid). This ensures that update in master will compensate for the increase in * memory on slave. */ - def reportBlockStatus( + private def reportBlockStatus( blockId: BlockId, info: BlockInfo, status: BlockStatus, - droppedMemorySize: Long = 0L) { + droppedMemorySize: Long = 0L): Unit = { val needReregister = !tryToReportBlockStatus(blockId, info, status, droppedMemorySize) if (needReregister) { - logInfo("Got told to re-register updating block " + blockId) + logInfo(s"Got told to re-register updating block $blockId") // Re-registering will report our new block for free. asyncReregister() } - logDebug("Told master about block " + blockId) + logDebug(s"Told master about block $blockId") } /** @@ -261,7 +280,9 @@ private[spark] class BlockManager( val onDiskSize = status.diskSize master.updateBlockInfo( blockManagerId, blockId, storageLevel, inMemSize, onDiskSize, inTachyonSize) - } else true + } else { + true + } } /** @@ -270,10 +291,10 @@ private[spark] class BlockManager( * and the updated in-memory and on-disk sizes. */ private def getCurrentBlockStatus(blockId: BlockId, info: BlockInfo): BlockStatus = { - val (newLevel, inMemSize, onDiskSize, inTachyonSize) = info.synchronized { + info.synchronized { info.level match { case null => - (StorageLevel.NONE, 0L, 0L, 0L) + BlockStatus(StorageLevel.NONE, 0L, 0L, 0L) case level => val inMem = level.useMemory && memoryStore.contains(blockId) val inTachyon = level.useOffHeap && tachyonStore.contains(blockId) @@ -284,19 +305,18 @@ private[spark] class BlockManager( val memSize = if (inMem) memoryStore.getSize(blockId) else 0L val tachyonSize = if (inTachyon) tachyonStore.getSize(blockId) else 0L val diskSize = if (onDisk) diskStore.getSize(blockId) else 0L - (storageLevel, memSize, diskSize, tachyonSize) + BlockStatus(storageLevel, memSize, diskSize, tachyonSize) } } - BlockStatus(newLevel, inMemSize, onDiskSize, inTachyonSize) } /** * Get locations of an array of blocks. */ - def getLocationBlockIds(blockIds: Array[BlockId]): Array[Seq[BlockManagerId]] = { + private def getLocationBlockIds(blockIds: Array[BlockId]): Array[Seq[BlockManagerId]] = { val startTimeMs = System.currentTimeMillis val locations = master.getLocations(blockIds).toArray - logDebug("Got multiple block location in " + Utils.getUsedTimeMs(startTimeMs)) + logDebug("Got multiple block location in %s".format(Utils.getUsedTimeMs(startTimeMs))) locations } @@ -306,15 +326,16 @@ private[spark] class BlockManager( * never deletes (recent) items. */ def getLocalFromDisk(blockId: BlockId, serializer: Serializer): Option[Iterator[Any]] = { - diskStore.getValues(blockId, serializer).orElse( - sys.error("Block " + blockId + " not found on disk, though it should be")) + diskStore.getValues(blockId, serializer).orElse { + throw new BlockException(blockId, s"Block $blockId not found on disk, though it should be") + } } /** * Get block from local block manager. */ def getLocal(blockId: BlockId): Option[Iterator[Any]] = { - logDebug("Getting local block " + blockId) + logDebug(s"Getting local block $blockId") doGetLocal(blockId, asValues = true).asInstanceOf[Option[Iterator[Any]]] } @@ -322,7 +343,7 @@ private[spark] class BlockManager( * Get block from the local block manager as serialized bytes. */ def getLocalBytes(blockId: BlockId): Option[ByteBuffer] = { - logDebug("Getting local block " + blockId + " as bytes") + logDebug(s"Getting local block $blockId as bytes") // 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) { @@ -330,7 +351,8 @@ private[spark] class BlockManager( case Some(bytes) => Some(bytes) case None => - throw new Exception("Block " + blockId + " not found on disk, though it should be") + throw new BlockException( + blockId, s"Block $blockId not found on disk, though it should be") } } else { doGetLocal(blockId, asValues = false).asInstanceOf[Option[ByteBuffer]] @@ -341,20 +363,29 @@ private[spark] class BlockManager( val info = blockInfo.get(blockId).orNull if (info != null) { info.synchronized { + // Double check to make sure the block is still there. There is a small chance that the + // block has been removed by removeBlock (which also synchronizes on the blockInfo object). + // Note that this only checks metadata tracking. If user intentionally deleted the block + // on disk or from off heap storage without using removeBlock, this conditional check will + // still pass but eventually we will get an exception because we can't find the block. + if (blockInfo.get(blockId).isEmpty) { + logWarning(s"Block $blockId had been removed") + return None + } // If another thread is writing the block, wait for it to become ready. if (!info.waitForReady()) { // If we get here, the block write failed. - logWarning("Block " + blockId + " was marked as failure.") + logWarning(s"Block $blockId was marked as failure.") return None } val level = info.level - logDebug("Level for block " + blockId + " is " + level) + logDebug(s"Level for block $blockId is $level") // Look for the block in memory if (level.useMemory) { - logDebug("Getting block " + blockId + " from memory") + logDebug(s"Getting block $blockId from memory") val result = if (asValues) { memoryStore.getValues(blockId) } else { @@ -364,51 +395,51 @@ private[spark] class BlockManager( case Some(values) => return Some(values) case None => - logDebug("Block " + blockId + " not found in memory") + logDebug(s"Block $blockId not found in memory") } } - + // Look for the block in Tachyon if (level.useOffHeap) { - logDebug("Getting block " + blockId + " from tachyon") + logDebug(s"Getting block $blockId from tachyon") if (tachyonStore.contains(blockId)) { tachyonStore.getBytes(blockId) match { - case Some(bytes) => { + case Some(bytes) => if (!asValues) { return Some(bytes) } else { return Some(dataDeserialize(blockId, bytes)) } - } case None => - logDebug("Block " + blockId + " not found in tachyon") + logDebug(s"Block $blockId not found in tachyon") } } } - // Look for block on disk, potentially storing it back into memory if required: + // Look for block on disk, potentially storing it back in memory if required if (level.useDisk) { - logDebug("Getting block " + blockId + " from disk") + logDebug(s"Getting block $blockId from disk") val bytes: ByteBuffer = diskStore.getBytes(blockId) match { - case Some(bytes) => bytes + case Some(b) => b case None => - throw new Exception("Block " + blockId + " not found on disk, though it should be") + throw new BlockException( + blockId, s"Block $blockId not found on disk, though it should be") } - assert (0 == bytes.position()) + assert(0 == bytes.position()) if (!level.useMemory) { - // If the block shouldn't be stored in memory, we can just return it: + // If the block shouldn't be stored in memory, we can just return it if (asValues) { return Some(dataDeserialize(blockId, bytes)) } else { return Some(bytes) } } else { - // Otherwise, we also have to store something in the memory store: + // Otherwise, we also have to store something in the memory store if (!level.deserialized || !asValues) { - // 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: + /* 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) @@ -419,16 +450,17 @@ private[spark] class BlockManager( } else { val values = dataDeserialize(blockId, bytes) if (level.deserialized) { - // Cache the values before returning them: + // Cache the values before returning them // TODO: Consider creating a putValues that also takes in a iterator? val valuesBuffer = new ArrayBuffer[Any] valuesBuffer ++= values - memoryStore.putValues(blockId, valuesBuffer, level, true).data match { - case Left(values2) => - return Some(values2) - case _ => - throw new Exception("Memory store did not return back an iterator") - } + memoryStore.putValues(blockId, valuesBuffer, level, returnValues = true).data + match { + case Left(values2) => + return Some(values2) + case _ => + throw new SparkException("Memory store did not return an iterator") + } } else { return Some(values) } @@ -437,7 +469,7 @@ private[spark] class BlockManager( } } } else { - logDebug("Block " + blockId + " not registered locally") + logDebug(s"Block $blockId not registered locally") } None } @@ -446,7 +478,7 @@ private[spark] class BlockManager( * Get block from remote block managers. */ def getRemote(blockId: BlockId): Option[Iterator[Any]] = { - logDebug("Getting remote block " + blockId) + logDebug(s"Getting remote block $blockId") doGetRemote(blockId, asValues = true).asInstanceOf[Option[Iterator[Any]]] } @@ -454,7 +486,7 @@ private[spark] class BlockManager( * Get block from remote block managers as serialized bytes. */ def getRemoteBytes(blockId: BlockId): Option[ByteBuffer] = { - logDebug("Getting remote block " + blockId + " as bytes") + logDebug(s"Getting remote block $blockId as bytes") doGetRemote(blockId, asValues = false).asInstanceOf[Option[ByteBuffer]] } @@ -462,7 +494,7 @@ private[spark] class BlockManager( require(blockId != null, "BlockId is null") val locations = Random.shuffle(master.getLocations(blockId)) for (loc <- locations) { - logDebug("Getting remote block " + blockId + " from " + loc) + logDebug(s"Getting remote block $blockId from $loc") val data = BlockManagerWorker.syncGetBlock( GetBlock(blockId), ConnectionManagerId(loc.host, loc.port)) if (data != null) { @@ -472,9 +504,9 @@ private[spark] class BlockManager( return Some(data) } } - logDebug("The value of block " + blockId + " is null") + logDebug(s"The value of block $blockId is null") } - logDebug("Block " + blockId + " not found") + logDebug(s"Block $blockId not found") None } @@ -484,12 +516,12 @@ private[spark] class BlockManager( def get(blockId: BlockId): Option[Iterator[Any]] = { val local = getLocal(blockId) if (local.isDefined) { - logInfo("Found block %s locally".format(blockId)) + logInfo(s"Found block $blockId locally") return local } val remote = getRemote(blockId) if (remote.isDefined) { - logInfo("Found block %s remotely".format(blockId)) + logInfo(s"Found block $blockId remotely") return remote } None @@ -510,7 +542,6 @@ private[spark] class BlockManager( } else { new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer) } - iter.initialize() iter } @@ -520,14 +551,14 @@ private[spark] class BlockManager( values: Iterator[Any], level: StorageLevel, tellMaster: Boolean): Seq[(BlockId, BlockStatus)] = { + require(values != null, "Values is null") doPut(blockId, IteratorValues(values), level, tellMaster) } /** * A short circuited method to get a block writer that can write data directly to disk. - * The Block will be appended to the File specified by filename. - * This is currently used for writing shuffle files out. Callers should handle error - * cases. + * The Block will be appended to the File specified by filename. This is currently used for + * writing shuffle files out. Callers should handle error cases. */ def getDiskWriter( blockId: BlockId, @@ -540,8 +571,8 @@ private[spark] class BlockManager( } /** - * Put a new block of values to the block manager. Return a list of blocks updated as a - * result of this put. + * Put a new block of values to the block manager. + * Return a list of blocks updated as a result of this put. */ def put( blockId: BlockId, @@ -553,8 +584,8 @@ private[spark] class BlockManager( } /** - * Put a new block of serialized bytes to the block manager. Return a list of blocks updated - * as a result of this put. + * Put a new block of serialized bytes to the block manager. + * Return a list of blocks updated as a result of this put. */ def putBytes( blockId: BlockId, @@ -567,7 +598,7 @@ private[spark] class BlockManager( private def doPut( blockId: BlockId, - data: Values, + data: BlockValues, level: StorageLevel, tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { @@ -577,20 +608,18 @@ private[spark] class BlockManager( // Return value val updatedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] - // Remember the block's storage level so that we can correctly drop it to disk if it needs - // to be dropped right after it got put into memory. Note, however, that other threads will - // not be able to get() this block until we call markReady on its BlockInfo. + /* Remember the block's storage level so that we can correctly drop it to disk if it needs + * to be dropped right after it got put into memory. Note, however, that other threads will + * not be able to get() this block until we call markReady on its BlockInfo. */ val putBlockInfo = { val tinfo = new BlockInfo(level, tellMaster) // Do atomically ! val oldBlockOpt = blockInfo.putIfAbsent(blockId, tinfo) - if (oldBlockOpt.isDefined) { if (oldBlockOpt.get.waitForReady()) { - logWarning("Block " + blockId + " already exists on this machine; not re-adding it") + logWarning(s"Block $blockId already exists on this machine; not re-adding it") return updatedBlocks } - // TODO: So the block info exists - but previous attempt to load it (?) failed. // What do we do now ? Retry on it ? oldBlockOpt.get @@ -601,10 +630,10 @@ private[spark] class BlockManager( val startTimeMs = System.currentTimeMillis - // If we're storing values and we need to replicate the data, we'll want access to the values, - // but because our put will read the whole iterator, there will be no values left. For the - // case where the put serializes data, we'll remember the bytes, above; but for the case where - // it doesn't, such as deserialized storage, let's rely on the put returning an Iterator. + /* If we're storing values and we need to replicate the data, we'll want access to the values, + * but because our put will read the whole iterator, there will be no values left. For the + * case where the put serializes data, we'll remember the bytes, above; but for the case where + * it doesn't, such as deserialized storage, let's rely on the put returning an Iterator. */ var valuesAfterPut: Iterator[Any] = null // Ditto for the bytes after the put @@ -615,81 +644,62 @@ private[spark] class BlockManager( // If we're storing bytes, then initiate the replication before storing them locally. // This is faster as data is already serialized and ready to send. - val replicationFuture = if (data.isInstanceOf[ByteBufferValues] && level.replication > 1) { - // Duplicate doesn't copy the bytes, just creates a wrapper - val bufferView = data.asInstanceOf[ByteBufferValues].buffer.duplicate() - Future { - replicate(blockId, bufferView, level) - } - } else { - null + val replicationFuture = data match { + case b: ByteBufferValues if level.replication > 1 => + // Duplicate doesn't copy the bytes, but just creates a wrapper + val bufferView = b.buffer.duplicate() + Future { replicate(blockId, bufferView, level) } + case _ => null } putBlockInfo.synchronized { - logTrace("Put for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs) - + " to get into synchronized block") + logTrace("Put for block %s took %s to get into synchronized block" + .format(blockId, Utils.getUsedTimeMs(startTimeMs))) var marked = false try { - if (level.useMemory) { - // Save it just to memory first, even if it also has useDisk set to true; we will - // drop it to disk later if the memory store can't hold it. - val res = data match { - case IteratorValues(iterator) => - memoryStore.putValues(blockId, iterator, level, true) - case ArrayBufferValues(array) => - memoryStore.putValues(blockId, array, level, true) - case ByteBufferValues(bytes) => { - bytes.rewind() - memoryStore.putBytes(blockId, bytes, level) - } - } - size = res.size - res.data match { - case Right(newBytes) => bytesAfterPut = newBytes - case Left(newIterator) => valuesAfterPut = newIterator - } - // Keep track of which blocks are dropped from memory - res.droppedBlocks.foreach { block => updatedBlocks += block } - } else if (level.useOffHeap) { - // Save to Tachyon. - val res = data match { - case IteratorValues(iterator) => - tachyonStore.putValues(blockId, iterator, level, false) - case ArrayBufferValues(array) => - tachyonStore.putValues(blockId, array, level, false) - case ByteBufferValues(bytes) => { - bytes.rewind(); - tachyonStore.putBytes(blockId, bytes, level) - } - } - size = res.size - res.data match { - case Right(newBytes) => bytesAfterPut = newBytes - case _ => - } - } else { - // Save directly to disk. - // Don't get back the bytes unless we replicate them. - val askForBytes = level.replication > 1 - - val res = data match { - case IteratorValues(iterator) => - diskStore.putValues(blockId, iterator, level, askForBytes) - case ArrayBufferValues(array) => - diskStore.putValues(blockId, array, level, askForBytes) - case ByteBufferValues(bytes) => { - bytes.rewind() - diskStore.putBytes(blockId, bytes, level) - } - } - size = res.size - res.data match { - case Right(newBytes) => bytesAfterPut = newBytes - case _ => + // returnValues - Whether to return the values put + // blockStore - The type of storage to put these values into + val (returnValues, blockStore: BlockStore) = { + if (level.useMemory) { + // Put it in memory first, even if it also has useDisk set to true; + // We will drop it to disk later if the memory store can't hold it. + (true, memoryStore) + } else if (level.useOffHeap) { + // Use tachyon for off-heap storage + (false, tachyonStore) + } else if (level.useDisk) { + // Don't get back the bytes from put unless we replicate them + (level.replication > 1, diskStore) + } else { + assert(level == StorageLevel.NONE) + throw new BlockException( + blockId, s"Attempted to put block $blockId without specifying storage level!") } } + // Actually put the values + val result = data match { + case IteratorValues(iterator) => + blockStore.putValues(blockId, iterator, level, returnValues) + case ArrayBufferValues(array) => + blockStore.putValues(blockId, array, level, returnValues) + case ByteBufferValues(bytes) => + bytes.rewind() + blockStore.putBytes(blockId, bytes, level) + } + size = result.size + result.data match { + case Left (newIterator) if level.useMemory => valuesAfterPut = newIterator + case Right (newBytes) => bytesAfterPut = newBytes + case _ => + } + + // Keep track of which blocks are dropped from memory + if (level.useMemory) { + result.droppedBlocks.foreach { updatedBlocks += _ } + } + val putBlockStatus = getCurrentBlockStatus(blockId, putBlockInfo) if (putBlockStatus.storageLevel != StorageLevel.NONE) { // Now that the block is in either the memory, tachyon, or disk store, @@ -709,18 +719,21 @@ private[spark] class BlockManager( // could've inserted a new BlockInfo before we remove it. blockInfo.remove(blockId) putBlockInfo.markFailure() - logWarning("Putting block " + blockId + " failed") + logWarning(s"Putting block $blockId failed") } } } - logDebug("Put block " + blockId + " locally took " + Utils.getUsedTimeMs(startTimeMs)) + logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) // Either we're storing bytes and we asynchronously started replication, or we're storing // values and need to serialize and replicate them now: if (level.replication > 1) { data match { - case ByteBufferValues(bytes) => Await.ready(replicationFuture, Duration.Inf) - case _ => { + case ByteBufferValues(bytes) => + if (replicationFuture != null) { + Await.ready(replicationFuture, Duration.Inf) + } + case _ => val remoteStartTime = System.currentTimeMillis // Serialize the block if not already done if (bytesAfterPut == null) { @@ -731,20 +744,19 @@ private[spark] class BlockManager( bytesAfterPut = dataSerialize(blockId, valuesAfterPut) } replicate(blockId, bytesAfterPut, level) - logDebug("Put block " + blockId + " remotely took " + - Utils.getUsedTimeMs(remoteStartTime)) - } + logDebug("Put block %s remotely took %s" + .format(blockId, Utils.getUsedTimeMs(remoteStartTime))) } } BlockManager.dispose(bytesAfterPut) if (level.replication > 1) { - logDebug("Put for block " + blockId + " with replication took " + - Utils.getUsedTimeMs(startTimeMs)) + logDebug("Putting block %s with replication took %s" + .format(blockId, Utils.getUsedTimeMs(startTimeMs))) } else { - logDebug("Put for block " + blockId + " without replication took " + - Utils.getUsedTimeMs(startTimeMs)) + logDebug("Putting block %s without replication took %s" + .format(blockId, Utils.getUsedTimeMs(startTimeMs))) } updatedBlocks @@ -753,8 +765,8 @@ private[spark] class BlockManager( /** * Replicate block to another node. */ - var cachedPeers: Seq[BlockManagerId] = null - private def replicate(blockId: BlockId, data: ByteBuffer, level: StorageLevel) { + @volatile var cachedPeers: Seq[BlockManagerId] = null + private def replicate(blockId: BlockId, data: ByteBuffer, level: StorageLevel): Unit = { val tLevel = StorageLevel( level.useDisk, level.useMemory, level.useOffHeap, level.deserialized, 1) if (cachedPeers == null) { @@ -763,15 +775,16 @@ private[spark] class BlockManager( for (peer: BlockManagerId <- cachedPeers) { val start = System.nanoTime data.rewind() - logDebug("Try to replicate BlockId " + blockId + " once; The size of the data is " - + data.limit() + " Bytes. To node: " + peer) - if (!BlockManagerWorker.syncPutBlock(PutBlock(blockId, data, tLevel), - new ConnectionManagerId(peer.host, peer.port))) { - logError("Failed to call syncPutBlock to " + peer) + logDebug(s"Try to replicate $blockId once; The size of the data is ${data.limit()} Bytes. " + + s"To node: $peer") + val putBlock = PutBlock(blockId, data, tLevel) + val cmId = new ConnectionManagerId(peer.host, peer.port) + val syncPutBlockSuccess = BlockManagerWorker.syncPutBlock(putBlock, cmId) + if (!syncPutBlockSuccess) { + logError(s"Failed to call syncPutBlock to $peer") } - logDebug("Replicated BlockId " + blockId + " once used " + - (System.nanoTime - start) / 1e6 + " s; The size of the data is " + - data.limit() + " bytes.") + logDebug("Replicating BlockId %s once used %fs; The size of the data is %d bytes." + .format(blockId, (System.nanoTime - start) / 1e6, data.limit())) } } @@ -803,17 +816,17 @@ private[spark] class BlockManager( blockId: BlockId, data: Either[ArrayBuffer[Any], ByteBuffer]): Option[BlockStatus] = { - logInfo("Dropping block " + blockId + " from memory") + logInfo(s"Dropping block $blockId from memory") val info = blockInfo.get(blockId).orNull // If the block has not already been dropped - if (info != null) { + if (info != null) { info.synchronized { // required ? As of now, this will be invoked only for blocks which are ready // But in case this changes in future, adding for consistency sake. if (!info.waitForReady()) { // If we get here, the block write failed. - logWarning("Block " + blockId + " was marked as failure. Nothing to drop") + logWarning(s"Block $blockId was marked as failure. Nothing to drop") return None } @@ -822,10 +835,10 @@ private[spark] class BlockManager( // Drop to disk, if storage level requires if (level.useDisk && !diskStore.contains(blockId)) { - logInfo("Writing block " + blockId + " to disk") + logInfo(s"Writing block $blockId to disk") data match { case Left(elements) => - diskStore.putValues(blockId, elements, level, false) + diskStore.putValues(blockId, elements, level, returnValues = false) case Right(bytes) => diskStore.putBytes(blockId, bytes, level) } @@ -839,7 +852,7 @@ private[spark] class BlockManager( if (blockIsRemoved) { blockIsUpdated = true } else { - logWarning("Block " + blockId + " could not be dropped from memory as it does not exist") + logWarning(s"Block $blockId could not be dropped from memory as it does not exist") } val status = getCurrentBlockStatus(blockId, info) @@ -863,69 +876,76 @@ private[spark] class BlockManager( * @return The number of blocks removed. */ def removeRdd(rddId: Int): Int = { - // TODO: Instead of doing a linear scan on the blockInfo map, create another map that maps - // from RDD.id to blocks. - logInfo("Removing RDD " + rddId) + // TODO: Avoid a linear scan by creating another mapping of RDD.id to blocks. + logInfo(s"Removing RDD $rddId") val blocksToRemove = blockInfo.keys.flatMap(_.asRDDId).filter(_.rddId == rddId) - blocksToRemove.foreach(blockId => removeBlock(blockId, tellMaster = false)) + blocksToRemove.foreach { blockId => removeBlock(blockId, tellMaster = false) } + blocksToRemove.size + } + + /** + * Remove all blocks belonging to the given broadcast. + */ + def removeBroadcast(broadcastId: Long, tellMaster: Boolean): Int = { + logInfo(s"Removing broadcast $broadcastId") + val blocksToRemove = blockInfo.keys.collect { + case bid @ BroadcastBlockId(`broadcastId`, _) => bid + } + blocksToRemove.foreach { blockId => removeBlock(blockId, tellMaster) } blocksToRemove.size } /** * Remove a block from both memory and disk. */ - def removeBlock(blockId: BlockId, tellMaster: Boolean = true) { - logInfo("Removing block " + blockId) + def removeBlock(blockId: BlockId, tellMaster: Boolean = true): Unit = { + logInfo(s"Removing block $blockId") val info = blockInfo.get(blockId).orNull - if (info != null) info.synchronized { - // 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) { - logWarning("Block " + blockId + " could not be removed as it was not found in either " + - "the disk, memory, or tachyon store") - } - blockInfo.remove(blockId) - if (tellMaster && info.tellMaster) { - val status = getCurrentBlockStatus(blockId, info) - reportBlockStatus(blockId, info, status) + if (info != null) { + info.synchronized { + // 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) { + logWarning(s"Block $blockId could not be removed as it was not found in either " + + "the disk, memory, or tachyon store") + } + blockInfo.remove(blockId) + if (tellMaster && info.tellMaster) { + val status = getCurrentBlockStatus(blockId, info) + reportBlockStatus(blockId, info, status) + } } } else { // The block has already been removed; do nothing. - logWarning("Asked to remove block " + blockId + ", which does not exist") + logWarning(s"Asked to remove block $blockId, which does not exist") } } - private def dropOldNonBroadcastBlocks(cleanupTime: Long) { - logInfo("Dropping non broadcast blocks older than " + cleanupTime) + private def dropOldNonBroadcastBlocks(cleanupTime: Long): Unit = { + logInfo(s"Dropping non broadcast blocks older than $cleanupTime") dropOldBlocks(cleanupTime, !_.isBroadcast) } - private def dropOldBroadcastBlocks(cleanupTime: Long) { - logInfo("Dropping broadcast blocks older than " + cleanupTime) + private def dropOldBroadcastBlocks(cleanupTime: Long): Unit = { + logInfo(s"Dropping broadcast blocks older than $cleanupTime") dropOldBlocks(cleanupTime, _.isBroadcast) } - private def dropOldBlocks(cleanupTime: Long, shouldDrop: (BlockId => Boolean)) { - val iterator = blockInfo.internalMap.entrySet().iterator() + private def dropOldBlocks(cleanupTime: Long, shouldDrop: (BlockId => Boolean)): Unit = { + val iterator = blockInfo.getEntrySet.iterator while (iterator.hasNext) { val entry = iterator.next() - val (id, info, time) = (entry.getKey, entry.getValue._1, entry.getValue._2) + val (id, info, time) = (entry.getKey, entry.getValue.value, entry.getValue.timestamp) if (time < cleanupTime && shouldDrop(id)) { info.synchronized { val level = info.level - if (level.useMemory) { - memoryStore.remove(id) - } - if (level.useDisk) { - diskStore.remove(id) - } - if (level.useOffHeap) { - tachyonStore.remove(id) - } + if (level.useMemory) { memoryStore.remove(id) } + if (level.useDisk) { diskStore.remove(id) } + if (level.useOffHeap) { tachyonStore.remove(id) } iterator.remove() - logInfo("Dropped block " + id) + logInfo(s"Dropped block $id") } val status = getCurrentBlockStatus(id, info) reportBlockStatus(id, info, status) @@ -933,12 +953,14 @@ private[spark] class BlockManager( } } - def shouldCompress(blockId: BlockId): Boolean = blockId match { - case ShuffleBlockId(_, _, _) => compressShuffle - case BroadcastBlockId(_) => compressBroadcast - case RDDBlockId(_, _) => compressRdds - case TempBlockId(_) => compressShuffleSpill - case _ => false + private def shouldCompress(blockId: BlockId): Boolean = { + blockId match { + case _: ShuffleBlockId => compressShuffle + case _: BroadcastBlockId => compressBroadcast + case _: RDDBlockId => compressRdds + case _: TempBlockId => compressShuffleSpill + case _ => false + } } /** @@ -960,8 +982,8 @@ private[spark] class BlockManager( blockId: BlockId, outputStream: OutputStream, values: Iterator[Any], - serializer: Serializer = defaultSerializer) { - val byteStream = new FastBufferedOutputStream(outputStream) + serializer: Serializer = defaultSerializer): Unit = { + val byteStream = new BufferedOutputStream(outputStream) val ser = serializer.newInstance() ser.serializeStream(wrapForCompression(blockId, byteStream)).writeAll(values).close() } @@ -971,10 +993,9 @@ private[spark] class BlockManager( blockId: BlockId, values: Iterator[Any], serializer: Serializer = defaultSerializer): ByteBuffer = { - val byteStream = new FastByteArrayOutputStream(4096) + val byteStream = new ByteArrayOutputStream(4096) dataSerializeStream(blockId, byteStream, values, serializer) - byteStream.trim() - ByteBuffer.wrap(byteStream.array) + ByteBuffer.wrap(byteStream.toByteArray) } /** @@ -986,21 +1007,41 @@ private[spark] class BlockManager( bytes: ByteBuffer, serializer: Serializer = defaultSerializer): Iterator[Any] = { bytes.rewind() - val stream = wrapForCompression(blockId, new ByteBufferInputStream(bytes, true)) - serializer.newInstance().deserializeStream(stream).asIterator + + def getIterator: Iterator[Any] = { + val stream = wrapForCompression(blockId, new ByteBufferInputStream(bytes, true)) + serializer.newInstance().deserializeStream(stream).asIterator + } + + if (blockId.isShuffle) { + /* Reducer may need to read many local shuffle blocks and will wrap them into Iterators + * at the beginning. The wrapping will cost some memory (compression instance + * initialization, etc.). Reducer reads shuffle blocks one by one so we could do the + * wrapping lazily to save memory. */ + class LazyProxyIterator(f: => Iterator[Any]) extends Iterator[Any] { + lazy val proxy = f + override def hasNext: Boolean = proxy.hasNext + override def next(): Any = proxy.next() + } + new LazyProxyIterator(getIterator) + } else { + getIterator + } } - def stop() { + def stop(): Unit = { if (heartBeatTask != null) { heartBeatTask.cancel() } connectionManager.stop() + shuffleBlockManager.stop() + diskBlockManager.stop() actorSystem.stop(slaveActor) blockInfo.clear() memoryStore.clear() diskStore.clear() if (tachyonInitialized) { - tachyonStore.clear() + tachyonStore.clear() } metadataCleaner.cancel() broadcastCleaner.cancel() @@ -1010,9 +1051,9 @@ private[spark] class BlockManager( private[spark] object BlockManager extends Logging { - val ID_GENERATOR = new IdGenerator + private val ID_GENERATOR = new IdGenerator - def getMaxMemory(conf: SparkConf): Long = { + private def getMaxMemory(conf: SparkConf): Long = { val memoryFraction = conf.getDouble("spark.storage.memoryFraction", 0.6) (Runtime.getRuntime.maxMemory * memoryFraction).toLong } @@ -1029,9 +1070,9 @@ private[spark] object BlockManager extends Logging { * waiting for the GC to find it because that could lead to huge numbers of open files. There's * unfortunately no standard API to do this. */ - def dispose(buffer: ByteBuffer) { + def dispose(buffer: ByteBuffer): Unit = { if (buffer != null && buffer.isInstanceOf[MappedByteBuffer]) { - logTrace("Unmapping " + buffer) + logTrace(s"Unmapping $buffer") if (buffer.asInstanceOf[DirectBuffer].cleaner() != null) { buffer.asInstanceOf[DirectBuffer].cleaner().clean() } @@ -1044,7 +1085,7 @@ private[spark] object BlockManager extends Logging { blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[BlockManagerId]] = { // blockManagerMaster != null is used in tests - assert (env != null || blockManagerMaster != null) + assert(env != null || blockManagerMaster != null) val blockLocations: Seq[Seq[BlockManagerId]] = if (blockManagerMaster == null) { env.blockManager.getLocationBlockIds(blockIds) } else { 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 be537d77309bc..b1585bd8199d1 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -20,17 +20,20 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} import java.util.concurrent.ConcurrentHashMap +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils /** + * :: DeveloperApi :: * This class represent an unique identifier for a BlockManager. - * The first 2 constructors of this class is made private to ensure that - * BlockManagerId objects can be created only using the apply method in - * the companion object. This allows de-duplication of ID objects. - * Also, constructor parameters are private to ensure that parameters cannot - * be modified from outside this class. + * + * The first 2 constructors of this class is made private to ensure that BlockManagerId objects + * can be created only using the apply method in the companion object. This allows de-duplication + * of ID objects. Also, constructor parameters are private to ensure that parameters cannot be + * modified from outside this class. */ -private[spark] class BlockManagerId private ( +@DeveloperApi +class BlockManagerId private ( private var executorId_ : String, private var host_ : String, private var port_ : Int, 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 4bc1b407ad106..7897fade2df2b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -81,6 +81,14 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log askDriverWithReply[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) = { + !getLocations(blockId).isEmpty + } + /** Get ids of other nodes in the cluster from the driver */ def getPeers(blockManagerId: BlockManagerId, numPeers: Int): Seq[BlockManagerId] = { val result = askDriverWithReply[Seq[BlockManagerId]](GetPeers(blockManagerId, numPeers)) @@ -99,12 +107,10 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log askDriverWithReply(RemoveBlock(blockId)) } - /** - * Remove all blocks belonging to the given RDD. - */ + /** Remove all blocks belonging to the given RDD. */ def removeRdd(rddId: Int, blocking: Boolean) { val future = askDriverWithReply[Future[Seq[Int]]](RemoveRdd(rddId)) - future onFailure { + future.onFailure { case e: Throwable => logError("Failed to remove RDD " + rddId, e) } if (blocking) { @@ -112,6 +118,31 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log } } + /** Remove all blocks belonging to the given shuffle. */ + def removeShuffle(shuffleId: Int, blocking: Boolean) { + val future = askDriverWithReply[Future[Seq[Boolean]]](RemoveShuffle(shuffleId)) + future.onFailure { + case e: Throwable => logError("Failed to remove shuffle " + shuffleId, e) + } + if (blocking) { + Await.result(future, timeout) + } + } + + /** Remove all blocks belonging to the given broadcast. */ + def removeBroadcast(broadcastId: Long, removeFromMaster: Boolean, blocking: Boolean) { + val future = askDriverWithReply[Future[Seq[Int]]]( + RemoveBroadcast(broadcastId, removeFromMaster)) + future.onFailure { + case e: Throwable => + logError("Failed to remove broadcast " + broadcastId + + " with removeFromMaster = " + removeFromMaster, e) + } + if (blocking) { + Await.result(future, timeout) + } + } + /** * Return the memory status for each block manager, in the form of a map from * the block manager's id to two long values. The first value is the maximum @@ -126,6 +157,51 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log askDriverWithReply[Array[StorageStatus]](GetStorageStatus) } + /** + * Return the block's status on all block managers, if any. NOTE: This is a + * potentially expensive operation and should only be used for testing. + * + * If askSlaves is true, this invokes the master to query each block manager for the most + * updated block statuses. This is useful when the master is not informed of the given block + * by all block managers. + */ + def getBlockStatus( + blockId: BlockId, + 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 + * 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. + */ + val response = askDriverWithReply[Map[BlockManagerId, Future[Option[BlockStatus]]]](msg) + val (blockManagerIds, futures) = response.unzip + val result = Await.result(Future.sequence(futures), timeout) + if (result == null) { + throw new SparkException("BlockManager returned null for BlockStatus query: " + blockId) + } + val blockStatus = result.asInstanceOf[Iterable[Option[BlockStatus]]] + blockManagerIds.zip(blockStatus).flatMap { case (blockManagerId, status) => + status.map { s => (blockManagerId, s) } + }.toMap + } + + /** + * Return a list of ids of existing blocks such that the ids match the given filter. NOTE: This + * is a potentially expensive operation and should only be used for testing. + * + * If askSlaves is true, this invokes the master to query each block manager for the most + * updated block statuses. This is useful when the master is not informed of the given block + * by all block managers. + */ + def getMatchingBlockIds( + filter: BlockId => Boolean, + askSlaves: Boolean): Seq[BlockId] = { + val msg = GetMatchingBlockIds(filter, askSlaves) + val future = askDriverWithReply[Future[Seq[BlockId]]](msg) + Await.result(future, timeout) + } + /** Stop the driver actor, called only on the Spark driver node */ def stop() { if (driverActor != null) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 378f4cadc17d7..6aed322eeb185 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -28,6 +28,7 @@ import akka.actor.{Actor, ActorRef, Cancellable} import akka.pattern.ask import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -94,9 +95,21 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus case GetStorageStatus => sender ! storageStatus + case GetBlockStatus(blockId, askSlaves) => + sender ! blockStatus(blockId, askSlaves) + + case GetMatchingBlockIds(filter, askSlaves) => + sender ! getMatchingBlockIds(filter, askSlaves) + case RemoveRdd(rddId) => sender ! removeRdd(rddId) + case RemoveShuffle(shuffleId) => + sender ! removeShuffle(shuffleId) + + case RemoveBroadcast(broadcastId, removeFromDriver) => + sender ! removeBroadcast(broadcastId, removeFromDriver) + case RemoveBlock(blockId) => removeBlockFromWorkers(blockId) sender ! true @@ -140,9 +153,41 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus // 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] - }.toSeq) + Future.sequence( + blockManagerInfo.values.map { bm => + bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Int] + }.toSeq + ) + } + + private def removeShuffle(shuffleId: Int): Future[Seq[Boolean]] = { + // Nothing to do in the BlockManagerMasterActor data structures + import context.dispatcher + val removeMsg = RemoveShuffle(shuffleId) + Future.sequence( + blockManagerInfo.values.map { bm => + bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Boolean] + }.toSeq + ) + } + + /** + * Delegate RemoveBroadcast messages to each BlockManager because the master may not notified + * of all broadcast blocks. If removeFromDriver is false, broadcast blocks are only removed + * from the executors, but not from the driver. + */ + private def removeBroadcast(broadcastId: Long, removeFromDriver: Boolean): Future[Seq[Int]] = { + // TODO: Consolidate usages of + import context.dispatcher + val removeMsg = RemoveBroadcast(broadcastId, removeFromDriver) + val requiredBlockManagers = blockManagerInfo.values.filter { info => + removeFromDriver || info.blockManagerId.executorId != "" + } + Future.sequence( + requiredBlockManagers.map { bm => + bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Int] + }.toSeq + ) } private def removeBlockManager(blockManagerId: BlockManagerId) { @@ -159,7 +204,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus val locations = blockLocations.get(blockId) locations -= blockManagerId if (locations.size == 0) { - blockLocations.remove(locations) + blockLocations.remove(blockId) } } listenerBus.post(SparkListenerBlockManagerRemoved(blockManagerId)) @@ -205,7 +250,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 ! RemoveBlock(blockId) + blockManager.get.slaveActor.ask(RemoveBlock(blockId))(akkaTimeout) } } } @@ -225,6 +270,61 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus }.toArray } + /** + * Return the block's status for all block managers, if any. NOTE: This is a + * potentially expensive operation and should only be used for testing. + * + * If askSlaves is true, the master queries each block manager for the most updated block + * statuses. This is useful when the master is not informed of the given block by all block + * managers. + */ + 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 + * 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. + */ + blockManagerInfo.values.map { info => + val blockStatusFuture = + if (askSlaves) { + info.slaveActor.ask(getBlockStatus)(akkaTimeout).mapTo[Option[BlockStatus]] + } else { + Future { info.getStatus(blockId) } + } + (info.blockManagerId, blockStatusFuture) + }.toMap + } + + /** + * Return the ids of blocks present in all the block managers that match the given filter. + * NOTE: This is a potentially expensive operation and should only be used for testing. + * + * If askSlaves is true, the master queries each block manager for the most updated block + * statuses. This is useful when the master is not informed of the given block by all block + * managers. + */ + 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]] + } else { + Future { info.blocks.keys.filter(filter).toSeq } + } + future + } + ).map(_.flatten.toSeq) + } + private def register(id: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { if (!blockManagerInfo.contains(id)) { blockManagerIdByExecutor.get(id.executorId) match { @@ -312,7 +412,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus } } -private[spark] case class BlockStatus( +@DeveloperApi +case class BlockStatus( storageLevel: StorageLevel, memSize: Long, diskSize: Long, @@ -334,6 +435,8 @@ private[spark] class BlockManagerInfo( logInfo("Registering block manager %s with %s RAM".format( blockManagerId.hostPort, Utils.bytesToString(maxMem))) + def getStatus(blockId: BlockId) = Option(_blocks.get(blockId)) + def updateLastSeenMs() { _lastSeenMs = System.currentTimeMillis() } 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 8a36b5cc42dfd..2b53bf33b5fba 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -34,6 +34,13 @@ private[storage] object BlockManagerMessages { // Remove all blocks belonging to a specific RDD. case class RemoveRdd(rddId: Int) extends ToBlockManagerSlave + // Remove all blocks belonging to a specific shuffle. + case class RemoveShuffle(shuffleId: Int) extends ToBlockManagerSlave + + // Remove all blocks belonging to a specific broadcast. + case class RemoveBroadcast(broadcastId: Long, removeFromDriver: Boolean = true) + extends ToBlockManagerSlave + ////////////////////////////////////////////////////////////////////////////////// // Messages from slaves to the master. @@ -80,7 +87,8 @@ private[storage] object BlockManagerMessages { } object UpdateBlockInfo { - def apply(blockManagerId: BlockManagerId, + def apply( + blockManagerId: BlockManagerId, blockId: BlockId, storageLevel: StorageLevel, memSize: Long, @@ -108,7 +116,13 @@ private[storage] object BlockManagerMessages { case object GetMemoryStatus extends ToBlockManagerMaster - case object ExpireDeadHosts extends ToBlockManagerMaster - case object GetStorageStatus extends ToBlockManagerMaster + + case class GetBlockStatus(blockId: BlockId, askSlaves: Boolean = true) + extends ToBlockManagerMaster + + case class GetMatchingBlockIds(filter: BlockId => Boolean, askSlaves: Boolean = true) + extends ToBlockManagerMaster + + case object ExpireDeadHosts extends ToBlockManagerMaster } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala index bcfb82d3c7336..6d4db064dff58 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala @@ -17,8 +17,11 @@ package org.apache.spark.storage -import akka.actor.Actor +import scala.concurrent.Future +import akka.actor.{ActorRef, Actor} + +import org.apache.spark.{Logging, MapOutputTracker} import org.apache.spark.storage.BlockManagerMessages._ /** @@ -26,14 +29,59 @@ import org.apache.spark.storage.BlockManagerMessages._ * this is used to remove blocks from the slave's BlockManager. */ private[storage] -class BlockManagerSlaveActor(blockManager: BlockManager) extends Actor { - override def receive = { +class BlockManagerSlaveActor( + blockManager: BlockManager, + mapOutputTracker: MapOutputTracker) + extends Actor with Logging { + + import context.dispatcher + // Operations that involve removing blocks may be slow and should be done asynchronously + override def receive = { case RemoveBlock(blockId) => - blockManager.removeBlock(blockId) + doAsync[Boolean]("removing block " + blockId, sender) { + blockManager.removeBlock(blockId) + true + } case RemoveRdd(rddId) => - val numBlocksRemoved = blockManager.removeRdd(rddId) - sender ! numBlocksRemoved + doAsync[Int]("removing RDD " + rddId, sender) { + blockManager.removeRdd(rddId) + } + + case RemoveShuffle(shuffleId) => + doAsync[Boolean]("removing shuffle " + shuffleId, sender) { + if (mapOutputTracker != null) { + mapOutputTracker.unregisterShuffle(shuffleId) + } + blockManager.shuffleBlockManager.removeShuffle(shuffleId) + } + + case RemoveBroadcast(broadcastId, tellMaster) => + doAsync[Int]("removing broadcast " + broadcastId, sender) { + blockManager.removeBroadcast(broadcastId, tellMaster) + } + + case GetBlockStatus(blockId, _) => + sender ! blockManager.getStatus(blockId) + + case GetMatchingBlockIds(filter, _) => + sender ! blockManager.getMatchingBlockIds(filter) + } + + private def doAsync[T](actionMessage: String, responseActor: ActorRef)(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) + } + future.onFailure { case t: Throwable => + logError("Error in " + actionMessage, t) + responseActor ! null.asInstanceOf[T] + } } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala index 7168ae18c2615..a2bfce7b4a0fa 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala @@ -37,7 +37,7 @@ private[spark] class BlockMessage() { private var id: BlockId = null private var data: ByteBuffer = null private var level: StorageLevel = null - + def set(getBlock: GetBlock) { typ = BlockMessage.TYPE_GET_BLOCK id = getBlock.id @@ -57,7 +57,6 @@ private[spark] class BlockMessage() { } def set(buffer: ByteBuffer) { - val startTime = System.currentTimeMillis /* println() println("BlockMessage: ") @@ -75,13 +74,13 @@ private[spark] class BlockMessage() { idBuilder += buffer.getChar() } id = BlockId(idBuilder.toString) - + if (typ == BlockMessage.TYPE_PUT_BLOCK) { val booleanInt = buffer.getInt() val replication = buffer.getInt() level = StorageLevel(booleanInt, replication) - + val dataLength = buffer.getInt() data = ByteBuffer.allocate(dataLength) if (dataLength != buffer.remaining) { @@ -100,7 +99,6 @@ private[spark] class BlockMessage() { data.flip() } - val finishTime = System.currentTimeMillis } def set(bufferMsg: BufferMessage) { @@ -108,14 +106,13 @@ private[spark] class BlockMessage() { buffer.clear() set(buffer) } - + def getType: Int = typ def getId: BlockId = id def getData: ByteBuffer = data def getLevel: StorageLevel = level - + def toBufferMessage: BufferMessage = { - val startTime = System.currentTimeMillis val buffers = new ArrayBuffer[ByteBuffer]() var buffer = ByteBuffer.allocate(4 + 4 + id.name.length * 2) buffer.putInt(typ).putInt(id.name.length) @@ -127,7 +124,7 @@ private[spark] class BlockMessage() { buffer = ByteBuffer.allocate(8).putInt(level.toInt).putInt(level.replication) buffer.flip() buffers += buffer - + buffer = ByteBuffer.allocate(4).putInt(data.remaining) buffer.flip() buffers += buffer @@ -140,7 +137,7 @@ private[spark] class BlockMessage() { buffers += data } - + /* println() println("BlockMessage: ") @@ -153,12 +150,11 @@ private[spark] class BlockMessage() { println() println() */ - val finishTime = System.currentTimeMillis Message.createBufferMessage(buffers) } override def toString: String = { - "BlockMessage [type = " + typ + ", id = " + id + ", level = " + level + + "BlockMessage [type = " + typ + ", id = " + id + ", level = " + level + ", data = " + (if (data != null) data.remaining.toString else "null") + "]" } } @@ -168,7 +164,7 @@ private[spark] object BlockMessage { val TYPE_GET_BLOCK: Int = 1 val TYPE_GOT_BLOCK: Int = 2 val TYPE_PUT_BLOCK: Int = 3 - + def fromBufferMessage(bufferMessage: BufferMessage): BlockMessage = { val newBlockMessage = new BlockMessage() newBlockMessage.set(bufferMessage) @@ -192,7 +188,7 @@ private[spark] object BlockMessage { newBlockMessage.set(gotBlock) newBlockMessage } - + def fromPutBlock(putBlock: PutBlock): BlockMessage = { val newBlockMessage = new BlockMessage() newBlockMessage.set(putBlock) @@ -206,7 +202,7 @@ private[spark] object BlockMessage { val bMsg = B.toBufferMessage val C = new BlockMessage() C.set(bMsg) - + println(B.getId + " " + B.getLevel) println(C.getId + " " + C.getLevel) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala index dc62b1efaa7d4..973d85c0a9b3a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala @@ -27,16 +27,16 @@ import org.apache.spark.network._ private[spark] class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockMessage] with Logging { - + def this(bm: BlockMessage) = this(Array(bm)) def this() = this(null.asInstanceOf[Seq[BlockMessage]]) - def apply(i: Int) = blockMessages(i) + def apply(i: Int) = blockMessages(i) def iterator = blockMessages.iterator - def length = blockMessages.length + def length = blockMessages.length def set(bufferMessage: BufferMessage) { val startTime = System.currentTimeMillis @@ -62,15 +62,15 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) logDebug("Trying to convert buffer " + newBuffer + " to block message") val newBlockMessage = BlockMessage.fromByteBuffer(newBuffer) logDebug("Created " + newBlockMessage) - newBlockMessages += newBlockMessage + newBlockMessages += newBlockMessage buffer.position(buffer.position() + size) } val finishTime = System.currentTimeMillis logDebug("Converted block message array from buffer message in " + (finishTime - startTime) / 1000.0 + " s") - this.blockMessages = newBlockMessages + this.blockMessages = newBlockMessages } - + def toBufferMessage: BufferMessage = { val buffers = new ArrayBuffer[ByteBuffer]() @@ -83,7 +83,7 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) buffers ++= bufferMessage.buffers logDebug("Added " + bufferMessage) }) - + logDebug("Buffer list:") buffers.foreach((x: ByteBuffer) => logDebug("" + x)) /* @@ -103,13 +103,13 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) } private[spark] object BlockMessageArray { - + def fromBufferMessage(bufferMessage: BufferMessage): BlockMessageArray = { val newBlockMessageArray = new BlockMessageArray() newBlockMessageArray.set(bufferMessage) newBlockMessageArray } - + def main(args: Array[String]) { val blockMessages = (0 until 10).map { i => @@ -124,10 +124,10 @@ private[spark] object BlockMessageArray { } val blockMessageArray = new BlockMessageArray(blockMessages) println("Block message array created") - + val bufferMessage = blockMessageArray.toBufferMessage println("Converted to buffer message") - + val totalSize = bufferMessage.size val newBuffer = ByteBuffer.allocate(totalSize) newBuffer.clear() @@ -137,7 +137,7 @@ private[spark] object BlockMessageArray { buffer.rewind() }) newBuffer.flip - val newBufferMessage = Message.createBufferMessage(newBuffer) + val newBufferMessage = Message.createBufferMessage(newBuffer) println("Copied to new buffer message, size = " + newBufferMessage.size) val newBlockMessageArray = BlockMessageArray.fromBufferMessage(newBufferMessage) @@ -147,7 +147,7 @@ private[spark] object BlockMessageArray { case BlockMessage.TYPE_PUT_BLOCK => { val pB = PutBlock(blockMessage.getId, blockMessage.getData, blockMessage.getLevel) println(pB) - } + } case BlockMessage.TYPE_GET_BLOCK => { val gB = new GetBlock(blockMessage.getId) println(gB) 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 696b930a26b9e..a2687e6be4e34 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -17,11 +17,9 @@ package org.apache.spark.storage -import java.io.{FileOutputStream, File, OutputStream} +import java.io.{BufferedOutputStream, FileOutputStream, File, OutputStream} import java.nio.channels.FileChannel -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream - import org.apache.spark.Logging import org.apache.spark.serializer.{SerializationStream, Serializer} @@ -119,7 +117,7 @@ private[spark] class DiskBlockObjectWriter( ts = new TimeTrackingOutputStream(fos) channel = fos.getChannel() lastValidPosition = initialPosition - bs = compressStream(new FastBufferedOutputStream(ts, bufferSize)) + bs = compressStream(new BufferedOutputStream(ts, bufferSize)) objOut = serializer.newInstance().serializeStream(bs) initialized = true this diff --git a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala index 9a9be047c7245..b9b53b1a2f118 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala @@ -24,11 +24,11 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.Logging /** - * Abstract class to store blocks + * Abstract class to store blocks. */ -private[spark] -abstract class BlockStore(val blockManager: BlockManager) extends Logging { - def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel) : PutResult +private[spark] abstract class BlockStore(val blockManager: BlockManager) extends Logging { + + def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel): PutResult /** * Put in a block and, possibly, also return its content as either bytes or another Iterator. @@ -37,11 +37,17 @@ abstract class BlockStore(val blockManager: BlockManager) extends Logging { * @return a PutResult that contains the size of the data, as well as the values put if * returnValues is true (if not, the result's data field can be null) */ - def putValues(blockId: BlockId, values: Iterator[Any], level: StorageLevel, - returnValues: Boolean) : PutResult + def putValues( + blockId: BlockId, + values: Iterator[Any], + level: StorageLevel, + returnValues: Boolean): PutResult - def putValues(blockId: BlockId, values: ArrayBuffer[Any], level: StorageLevel, - returnValues: Boolean) : PutResult + def putValues( + blockId: BlockId, + values: ArrayBuffer[Any], + level: StorageLevel, + returnValues: Boolean): PutResult /** * Return the size of a block in bytes. 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 f3e1c38744d78..2ec46d416f37d 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -40,9 +40,9 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 private val subDirsPerLocalDir = shuffleManager.conf.getInt("spark.diskStore.subDirectories", 64) - // Create one local directory for each path mentioned in spark.local.dir; 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. + /* Create one local directory for each path mentioned in spark.local.dir; 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. */ private val localDirs: Array[File] = createLocalDirs() private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) private var shuffleSender : ShuffleSender = null @@ -90,6 +90,20 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD def getFile(blockId: BlockId): File = getFile(blockId.name) + /** Check if disk block manager has a block. */ + def containsBlock(blockId: BlockId): Boolean = { + getBlockLocation(blockId).file.exists() + } + + /** List all the blocks currently stored on disk by the disk manager. */ + def getAllBlocks(): Seq[BlockId] = { + // Get all the files inside the array of array of directories + subDirs.flatten.filter(_ != null).flatMap { dir => + val files = dir.list() + if (files != null) files else Seq.empty + }.map(BlockId.apply) + } + /** Produces a unique block id and File suitable for intermediate results. */ def createTempBlock(): (TempBlockId, File) = { var blockId = new TempBlockId(UUID.randomUUID()) @@ -100,7 +114,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD } private def createLocalDirs(): Array[File] = { - logDebug("Creating local directories at root dirs '" + rootDirs + "'") + logDebug(s"Creating local directories at root dirs '$rootDirs'") val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss") rootDirs.split(",").map { rootDir => var foundLocalDir = false @@ -112,21 +126,20 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD tries += 1 try { localDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536)) - localDir = new File(rootDir, "spark-local-" + localDirId) + localDir = new File(rootDir, s"spark-local-$localDirId") if (!localDir.exists) { foundLocalDir = localDir.mkdirs() } } catch { case e: Exception => - logWarning("Attempt " + tries + " to create local dir " + localDir + " failed", e) + logWarning(s"Attempt $tries to create local dir $localDir failed", e) } } if (!foundLocalDir) { - logError("Failed " + MAX_DIR_CREATION_ATTEMPTS + - " attempts to create local dir in " + rootDir) + logError(s"Failed $MAX_DIR_CREATION_ATTEMPTS attempts to create local dir in $rootDir") System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) } - logInfo("Created local directory at " + localDir) + logInfo(s"Created local directory at $localDir") localDir } } @@ -134,27 +147,34 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD private def addShutdownHook() { localDirs.foreach(localDir => Utils.registerShutdownDeleteDir(localDir)) Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { logDebug("Shutdown hook called") - localDirs.foreach { localDir => - try { - if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) - } catch { - case t: Throwable => - logError("Exception while deleting local spark dir: " + localDir, t) - } - } + DiskBlockManager.this.stop() + } + }) + } - if (shuffleSender != null) { - shuffleSender.stop() + /** Cleanup local dirs and stop shuffle sender. */ + private[spark] def stop() { + localDirs.foreach { localDir => + if (localDir.isDirectory() && localDir.exists()) { + try { + if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) + } catch { + case e: Exception => + logError(s"Exception while deleting local spark dir: $localDir", e) } } - }) + } + + if (shuffleSender != null) { + shuffleSender.stop() + } } private[storage] def startShuffleBlockSender(port: Int): Int = { shuffleSender = new ShuffleSender(port, this) - logInfo("Created ShuffleSender binding to port : " + shuffleSender.port) + logInfo(s"Created ShuffleSender binding to port: ${shuffleSender.port}") shuffleSender.port } } 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 36ee4bcc41c66..ebff0cb5ba153 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -33,45 +33,45 @@ import org.apache.spark.util.Utils private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManager) extends BlockStore(blockManager) with Logging { + val minMemoryMapBytes = blockManager.conf.getLong("spark.storage.memoryMapThreshold", 2 * 4096L) + override def getSize(blockId: BlockId): Long = { diskManager.getBlockLocation(blockId).length } - override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel) : PutResult = { + override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): PutResult = { // So that we do not modify the input offsets ! // duplicate does not copy buffer, so inexpensive val bytes = _bytes.duplicate() - logDebug("Attempting to put block " + blockId) + logDebug(s"Attempting to put block $blockId") val startTime = System.currentTimeMillis val file = diskManager.getFile(blockId) - val channel = new FileOutputStream(file).getChannel() + val channel = new FileOutputStream(file).getChannel while (bytes.remaining > 0) { channel.write(bytes) } 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))) - return PutResult(bytes.limit(), Right(bytes.duplicate())) + file.getName, Utils.bytesToString(bytes.limit), finishTime - startTime)) + PutResult(bytes.limit(), Right(bytes.duplicate())) } override def putValues( blockId: BlockId, values: ArrayBuffer[Any], level: StorageLevel, - returnValues: Boolean) - : PutResult = { - return putValues(blockId, values.toIterator, level, returnValues) + returnValues: Boolean): PutResult = { + putValues(blockId, values.toIterator, level, returnValues) } override def putValues( blockId: BlockId, values: Iterator[Any], level: StorageLevel, - returnValues: Boolean) - : PutResult = { + returnValues: Boolean): PutResult = { - logDebug("Attempting to write values for block " + blockId) + logDebug(s"Attempting to write values for block $blockId") val startTime = System.currentTimeMillis val file = diskManager.getFile(blockId) val outputStream = new FileOutputStream(file) @@ -93,13 +93,21 @@ private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManage override def getBytes(blockId: BlockId): Option[ByteBuffer] = { val segment = diskManager.getBlockLocation(blockId) - val channel = new RandomAccessFile(segment.file, "r").getChannel() - val buffer = try { - channel.map(MapMode.READ_ONLY, segment.offset, segment.length) + val channel = new RandomAccessFile(segment.file, "r").getChannel + + try { + // For small files, directly read rather than memory map + if (segment.length < minMemoryMapBytes) { + val buf = ByteBuffer.allocate(segment.length.toInt) + channel.read(buf, segment.offset) + buf.flip() + Some(buf) + } else { + Some(channel.map(MapMode.READ_ONLY, segment.offset, segment.length)) + } } finally { channel.close() } - Some(buffer) } override def getValues(blockId: BlockId): Option[Iterator[Any]] = { @@ -121,7 +129,7 @@ private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManage file.delete() } else { if (fileSegment.length < file.length()) { - logWarning("Could not delete block associated with only a part of a file: " + blockId) + logWarning(s"Could not delete block associated with only a part of a file: $blockId") } false } 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 555486830a769..132502b75f8cd 100644 --- a/core/src/main/scala/org/apache/spark/storage/FileSegment.scala +++ b/core/src/main/scala/org/apache/spark/storage/FileSegment.scala @@ -23,6 +23,6 @@ import java.io.File * References a particular segment of a file (potentially the entire file), * based off an offset and a length. */ -private[spark] class FileSegment(val file: File, val offset: Long, val length : Long) { +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) } 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 488f1ea9628f5..71f66c826c5b3 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -24,6 +24,8 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.util.{SizeEstimator, Utils} +private case class MemoryEntry(value: Any, size: Long, deserialized: Boolean) + /** * Stores blocks in memory, either as ArrayBuffers of deserialized Java objects or as * serialized ByteBuffers. @@ -31,15 +33,13 @@ import org.apache.spark.util.{SizeEstimator, Utils} private class MemoryStore(blockManager: BlockManager, maxMemory: Long) extends BlockStore(blockManager) { - case class Entry(value: Any, size: Long, deserialized: Boolean) - - private val entries = new LinkedHashMap[BlockId, Entry](32, 0.75f, true) + private val entries = new LinkedHashMap[BlockId, MemoryEntry](32, 0.75f, true) @volatile private var currentMemory = 0L // Object used to ensure that only one thread is putting blocks and if necessary, dropping // blocks from the memory store. private val putLock = new Object() - logInfo("MemoryStore started with capacity %s.".format(Utils.bytesToString(maxMemory))) + logInfo("MemoryStore started with capacity %s".format(Utils.bytesToString(maxMemory))) def freeMemory: Long = maxMemory - currentMemory @@ -58,11 +58,11 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) val elements = new ArrayBuffer[Any] elements ++= values val sizeEstimate = SizeEstimator.estimate(elements.asInstanceOf[AnyRef]) - tryToPut(blockId, elements, sizeEstimate, true) - PutResult(sizeEstimate, Left(values.toIterator)) + val putAttempt = tryToPut(blockId, elements, sizeEstimate, deserialized = true) + PutResult(sizeEstimate, Left(values.toIterator), putAttempt.droppedBlocks) } else { - tryToPut(blockId, bytes, bytes.limit, false) - PutResult(bytes.limit(), Right(bytes.duplicate())) + val putAttempt = tryToPut(blockId, bytes, bytes.limit, deserialized = false) + PutResult(bytes.limit(), Right(bytes.duplicate()), putAttempt.droppedBlocks) } } @@ -101,7 +101,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } else if (entry.deserialized) { Some(blockManager.dataSerialize(blockId, entry.value.asInstanceOf[ArrayBuffer[Any]].iterator)) } else { - Some(entry.value.asInstanceOf[ByteBuffer].duplicate()) // Doesn't actually copy the data + Some(entry.value.asInstanceOf[ByteBuffer].duplicate()) // Doesn't actually copy the data } } @@ -124,8 +124,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) val entry = entries.remove(blockId) if (entry != null) { currentMemory -= entry.size - logInfo("Block %s of size %d dropped from memory (free %d)".format( - blockId, entry.size, freeMemory)) + logInfo(s"Block $blockId of size ${entry.size} dropped from memory (free $freeMemory)") true } else { false @@ -181,18 +180,14 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) droppedBlocks ++= freeSpaceResult.droppedBlocks if (enoughFreeSpace) { - val entry = new Entry(value, size, deserialized) + val entry = new MemoryEntry(value, size, deserialized) entries.synchronized { entries.put(blockId, entry) currentMemory += size } - if (deserialized) { - logInfo("Block %s stored as values to memory (estimated size %s, free %s)".format( - blockId, Utils.bytesToString(size), Utils.bytesToString(freeMemory))) - } else { - logInfo("Block %s stored as bytes to memory (size %s, free %s)".format( - blockId, Utils.bytesToString(size), Utils.bytesToString(freeMemory))) - } + val valuesOrBytes = if (deserialized) "values" else "bytes" + logInfo("Block %s stored as %s in memory (estimated size %s, free %s)".format( + blockId, valuesOrBytes, Utils.bytesToString(size), Utils.bytesToString(freeMemory))) putSuccess = true } else { // Tell the block manager that we couldn't put it in memory so that it can drop it to @@ -221,13 +216,12 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) * Return whether there is enough free space, along with the blocks dropped in the process. */ private def ensureFreeSpace(blockIdToAdd: BlockId, space: Long): ResultWithDroppedBlocks = { - logInfo("ensureFreeSpace(%d) called with curMem=%d, maxMem=%d".format( - space, currentMemory, maxMemory)) + logInfo(s"ensureFreeSpace($space) called with curMem=$currentMemory, maxMem=$maxMemory") val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] if (space > maxMemory) { - logInfo("Will not store " + blockIdToAdd + " as it is larger than our memory limit") + logInfo(s"Will not store $blockIdToAdd as it is larger than our memory limit") return ResultWithDroppedBlocks(success = false, droppedBlocks) } @@ -252,7 +246,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } if (maxMemory - (currentMemory - selectedMemory) >= space) { - logInfo(selectedBlocks.size + " blocks selected for dropping") + logInfo(s"${selectedBlocks.size} blocks selected for dropping") for (blockId <- selectedBlocks) { val entry = entries.synchronized { entries.get(blockId) } // This should never be null as only one thread should be dropping diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala new file mode 100644 index 0000000000000..5a72e216872a6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.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.storage + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils + +@DeveloperApi +class RDDInfo( + val id: Int, + val name: String, + val numPartitions: Int, + var storageLevel: StorageLevel) + extends Ordered[RDDInfo] { + + var numCachedPartitions = 0 + var memSize = 0L + var diskSize = 0L + var tachyonSize = 0L + + override def toString = { + import Utils.bytesToString + ("RDD \"%s\" (%d) StorageLevel: %s; CachedPartitions: %d; TotalPartitions: %d; " + + "MemorySize: %s; TachyonSize: %s; DiskSize: %s").format( + name, id, storageLevel.toString, numCachedPartitions, numPartitions, + bytesToString(memSize), bytesToString(tachyonSize), bytesToString(diskSize)) + } + + override def compare(that: RDDInfo) = { + 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) + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index bb07c8cb134cc..35910e552fe86 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -169,23 +169,47 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { throw new IllegalStateException("Failed to find shuffle block: " + id) } + /** Remove all the blocks / files and metadata related to a particular shuffle. */ + def removeShuffle(shuffleId: ShuffleId): Boolean = { + // Do not change the ordering of this, if shuffleStates should be removed only + // after the corresponding shuffle blocks have been removed + val cleaned = removeShuffleBlocks(shuffleId) + shuffleStates.remove(shuffleId) + cleaned + } + + /** Remove all the blocks / files related to a particular shuffle. */ + private def removeShuffleBlocks(shuffleId: ShuffleId): Boolean = { + shuffleStates.get(shuffleId) match { + case Some(state) => + if (consolidateShuffleFiles) { + for (fileGroup <- state.allFileGroups; file <- fileGroup.files) { + file.delete() + } + } else { + for (mapId <- state.completedMapTasks; reduceId <- 0 until state.numBuckets) { + val blockId = new ShuffleBlockId(shuffleId, mapId, reduceId) + blockManager.diskBlockManager.getFile(blockId).delete() + } + } + logInfo("Deleted all files for shuffle " + shuffleId) + true + case None => + logInfo("Could not find files for shuffle " + shuffleId + " for deleting") + false + } + } + private def physicalFileName(shuffleId: Int, bucketId: Int, fileId: Int) = { "merged_shuffle_%d_%d_%d".format(shuffleId, bucketId, fileId) } private def cleanup(cleanupTime: Long) { - shuffleStates.clearOldValues(cleanupTime, (shuffleId, state) => { - if (consolidateShuffleFiles) { - for (fileGroup <- state.allFileGroups; file <- fileGroup.files) { - file.delete() - } - } else { - for (mapId <- state.completedMapTasks; reduceId <- 0 until state.numBuckets) { - val blockId = new ShuffleBlockId(shuffleId, mapId, reduceId) - blockManager.diskBlockManager.getFile(blockId).delete() - } - } - }) + shuffleStates.clearOldValues(cleanupTime, (shuffleId, state) => removeShuffleBlocks(shuffleId)) + } + + def stop() { + metadataCleaner.cancel() } } 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 95e71de2d3f1d..1e35abaab5353 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -19,21 +19,26 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} +import org.apache.spark.annotation.DeveloperApi + /** + * :: 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. + * * 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 * factory method of the singleton object (`StorageLevel(...)`). */ +@DeveloperApi class StorageLevel private( - private var useDisk_ : Boolean, - private var useMemory_ : Boolean, - private var useOffHeap_ : Boolean, - private var deserialized_ : Boolean, - private var replication_ : Int = 1) + private var _useDisk: Boolean, + private var _useMemory: Boolean, + private var _useOffHeap: Boolean, + private var _deserialized: Boolean, + private var _replication: Int = 1) extends Externalizable { // TODO: Also add fields for caching priority, dataset ID, and flushing. @@ -43,23 +48,24 @@ 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 = _useDisk + def useMemory = _useMemory + def useOffHeap = _useOffHeap + def deserialized = _deserialized + def replication = _replication - assert(replication < 40, "Replication restricted to be less than 40 for calculating hashcodes") + assert(replication < 40, "Replication restricted to be less than 40 for calculating hash codes") if (useOffHeap) { - require(useDisk == false, "Off-heap storage level does not support using disk") - require(useMemory == false, "Off-heap storage level does not support using heap memory") - require(deserialized == false, "Off-heap storage level does not support deserialized storage") + require(!useDisk, "Off-heap storage level does not support using disk") + require(!useMemory, "Off-heap storage level does not support using heap memory") + require(!deserialized, "Off-heap storage level does not support deserialized storage") require(replication == 1, "Off-heap storage level does not support multiple replication") } - override def clone(): StorageLevel = new StorageLevel( - this.useDisk, this.useMemory, this.useOffHeap, this.deserialized, this.replication) + override def clone(): StorageLevel = { + new StorageLevel(useDisk, useMemory, useOffHeap, deserialized, replication) + } override def equals(other: Any): Boolean = other match { case s: StorageLevel => @@ -72,20 +78,20 @@ class StorageLevel private( false } - def isValid = ((useMemory || useDisk || useOffHeap) && (replication > 0)) + def isValid = (useMemory || useDisk || useOffHeap) && (replication > 0) def toInt: Int = { var ret = 0 - if (useDisk_) { + if (_useDisk) { ret |= 8 } - if (useMemory_) { + if (_useMemory) { ret |= 4 } - if (useOffHeap_) { + if (_useOffHeap) { ret |= 2 } - if (deserialized_) { + if (_deserialized) { ret |= 1 } ret @@ -93,32 +99,34 @@ class StorageLevel private( override def writeExternal(out: ObjectOutput) { out.writeByte(toInt) - out.writeByte(replication_) + out.writeByte(_replication) } override def readExternal(in: ObjectInput) { val flags = in.readByte() - useDisk_ = (flags & 8) != 0 - useMemory_ = (flags & 4) != 0 - useOffHeap_ = (flags & 2) != 0 - deserialized_ = (flags & 1) != 0 - replication_ = in.readByte() + _useDisk = (flags & 8) != 0 + _useMemory = (flags & 4) != 0 + _useOffHeap = (flags & 2) != 0 + _deserialized = (flags & 1) != 0 + _replication = in.readByte() } @throws(classOf[IOException]) private def readResolve(): Object = StorageLevel.getCachedStorageLevel(this) - override def toString: String = "StorageLevel(%b, %b, %b, %b, %d)".format( - useDisk, useMemory, useOffHeap, deserialized, replication) + override def toString: String = { + s"StorageLevel($useDisk, $useMemory, $useOffHeap, $deserialized, $replication)" + } override def hashCode(): Int = toInt * 41 + replication - def description : String = { + + def description: String = { var result = "" result += (if (useDisk) "Disk " else "") result += (if (useMemory) "Memory " else "") result += (if (useOffHeap) "Tachyon " else "") result += (if (deserialized) "Deserialized " else "Serialized ") - result += "%sx Replicated".format(replication) + result += s"${replication}x Replicated" result } } @@ -142,29 +150,77 @@ object StorageLevel { val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2) val OFF_HEAP = new StorageLevel(false, false, true, false) - /** Create a new StorageLevel object without setting useOffHeap */ - def apply(useDisk: Boolean, useMemory: Boolean, useOffHeap: Boolean, - deserialized: Boolean, replication: Int) = getCachedStorageLevel( + /** + * :: DeveloperApi :: + * Return the StorageLevel object with the specified name. + */ + @DeveloperApi + def fromString(s: String): StorageLevel = s match { + case "NONE" => NONE + case "DISK_ONLY" => DISK_ONLY + case "DISK_ONLY_2" => DISK_ONLY_2 + case "MEMORY_ONLY" => MEMORY_ONLY + case "MEMORY_ONLY_2" => MEMORY_ONLY_2 + case "MEMORY_ONLY_SER" => MEMORY_ONLY_SER + case "MEMORY_ONLY_SER_2" => MEMORY_ONLY_SER_2 + case "MEMORY_AND_DISK" => MEMORY_AND_DISK + case "MEMORY_AND_DISK_2" => MEMORY_AND_DISK_2 + case "MEMORY_AND_DISK_SER" => MEMORY_AND_DISK_SER + case "MEMORY_AND_DISK_SER_2" => MEMORY_AND_DISK_SER_2 + case "OFF_HEAP" => OFF_HEAP + case _ => throw new IllegalArgumentException(s"Invalid StorageLevel: $s") + } + + /** + * :: DeveloperApi :: + * Create a new StorageLevel object without setting useOffHeap. + */ + @DeveloperApi + def apply( + useDisk: Boolean, + useMemory: Boolean, + useOffHeap: Boolean, + deserialized: Boolean, + replication: Int) = { + getCachedStorageLevel( new StorageLevel(useDisk, useMemory, useOffHeap, deserialized, replication)) + } - /** Create a new StorageLevel object */ - def apply(useDisk: Boolean, useMemory: Boolean, - deserialized: Boolean, replication: Int = 1) = getCachedStorageLevel( - new StorageLevel(useDisk, useMemory, false, deserialized, replication)) + /** + * :: DeveloperApi :: + * Create a new StorageLevel object. + */ + @DeveloperApi + def apply( + useDisk: Boolean, + useMemory: Boolean, + deserialized: Boolean, + replication: Int = 1) = { + getCachedStorageLevel(new StorageLevel(useDisk, useMemory, false, deserialized, replication)) + } - /** Create a new StorageLevel object from its integer representation */ - def apply(flags: Int, replication: Int): StorageLevel = + /** + * :: DeveloperApi :: + * Create a new StorageLevel object from its integer representation. + */ + @DeveloperApi + def apply(flags: Int, replication: Int): StorageLevel = { getCachedStorageLevel(new StorageLevel(flags, replication)) + } - /** Read StorageLevel object from ObjectInput stream */ + /** + * :: DeveloperApi :: + * Read StorageLevel object from ObjectInput stream. + */ + @DeveloperApi def apply(in: ObjectInput): StorageLevel = { val obj = new StorageLevel() obj.readExternal(in) getCachedStorageLevel(obj) } - private[spark] - val storageLevelCache = new java.util.concurrent.ConcurrentHashMap[StorageLevel, StorageLevel]() + private[spark] val storageLevelCache = + new java.util.concurrent.ConcurrentHashMap[StorageLevel, StorageLevel]() private[spark] def getCachedStorageLevel(level: StorageLevel): StorageLevel = { storageLevelCache.putIfAbsent(level, level) 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 7a174959037be..c694fc8c347ec 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -19,12 +19,15 @@ package org.apache.spark.storage import scala.collection.mutable +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ /** - * A SparkListener that maintains executor storage status + * :: DeveloperApi :: + * A SparkListener that maintains executor storage status. */ -private[spark] class StorageStatusListener extends SparkListener { +@DeveloperApi +class StorageStatusListener extends SparkListener { private val executorIdToStorageStatus = mutable.Map[String, StorageStatus]() def storageStatusList = executorIdToStorageStatus.values.toSeq @@ -34,7 +37,11 @@ private[spark] class StorageStatusListener extends SparkListener { val filteredStatus = storageStatusList.find(_.blockManagerId.executorId == execId) filteredStatus.foreach { storageStatus => updatedBlocks.foreach { case (blockId, updatedStatus) => - storageStatus.blocks(blockId) = updatedStatus + if (updatedStatus.storageLevel == StorageLevel.NONE) { + storageStatus.blocks.remove(blockId) + } else { + storageStatus.blocks(blockId) = updatedStatus + } } } } 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 ff6e84cf9819a..f3bde1df45c79 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -21,60 +21,35 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.SparkContext -import org.apache.spark.util.Utils +import org.apache.spark.annotation.DeveloperApi -private[spark] +/** + * :: DeveloperApi :: + * Storage information for each BlockManager. + */ +@DeveloperApi class StorageStatus( val blockManagerId: BlockManagerId, val maxMem: Long, val blocks: mutable.Map[BlockId, BlockStatus] = mutable.Map.empty) { - def memUsed() = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) + def memUsed = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) def memUsedByRDD(rddId: Int) = rddBlocks.filterKeys(_.rddId == rddId).values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) - def diskUsed() = blocks.values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) + def diskUsed = blocks.values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) def diskUsedByRDD(rddId: Int) = rddBlocks.filterKeys(_.rddId == rddId).values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) - def memRemaining : Long = maxMem - memUsed() - - def rddBlocks = blocks.flatMap { - case (rdd: RDDBlockId, status) => Some(rdd, status) - case _ => None - } -} + def memRemaining: Long = maxMem - memUsed -private[spark] -class RDDInfo( - val id: Int, - val name: String, - val numPartitions: Int, - val storageLevel: StorageLevel) extends Ordered[RDDInfo] { - - var numCachedPartitions = 0 - var memSize = 0L - var diskSize = 0L - var tachyonSize= 0L - - override def toString = { - import Utils.bytesToString - ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s;" + - "TachyonSize: %s; DiskSize: %s").format( - name, id, storageLevel.toString, numCachedPartitions, numPartitions, - bytesToString(memSize), bytesToString(tachyonSize), bytesToString(diskSize)) - } - - override def compare(that: RDDInfo) = { - this.id - that.id - } + def rddBlocks = blocks.collect { case (rdd: RDDBlockId, status) => (rdd, status) } } -/* Helper methods for storage-related objects */ -private[spark] -object StorageUtils { +/** Helper methods for storage-related objects. */ +private[spark] object StorageUtils { /** * Returns basic information of all RDDs persisted in the given SparkContext. This does not @@ -114,10 +89,13 @@ object StorageUtils { // Add up memory, disk and Tachyon sizes val persistedBlocks = blocks.filter { status => status.memSize + status.diskSize + status.tachyonSize > 0 } + val _storageLevel = + if (persistedBlocks.length > 0) persistedBlocks(0).storageLevel else StorageLevel.NONE val memSize = persistedBlocks.map(_.memSize).reduceOption(_ + _).getOrElse(0L) val diskSize = persistedBlocks.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) val tachyonSize = persistedBlocks.map(_.tachyonSize).reduceOption(_ + _).getOrElse(0L) rddInfoMap.get(rddId).map { rddInfo => + rddInfo.storageLevel = _storageLevel rddInfo.numCachedPartitions = persistedBlocks.length rddInfo.memSize = memSize rddInfo.diskSize = diskSize 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 b0b9674856568..a6cbe3aa440ff 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala @@ -25,7 +25,6 @@ import tachyon.client.TachyonFile import org.apache.spark.Logging import org.apache.spark.executor.ExecutorExitCode -import org.apache.spark.network.netty.ShuffleSender import org.apache.spark.util.Utils @@ -137,7 +136,7 @@ private[spark] class TachyonBlockManager( private def addShutdownHook() { tachyonDirs.foreach(tachyonDir => Utils.registerShutdownDeleteDir(tachyonDir)) Runtime.getRuntime.addShutdownHook(new Thread("delete Spark tachyon dirs") { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { logDebug("Shutdown hook called") tachyonDirs.foreach { tachyonDir => try { @@ -145,8 +144,8 @@ private[spark] class TachyonBlockManager( Utils.deleteRecursively(tachyonDir, client) } } catch { - case t: Throwable => - logError("Exception while deleting tachyon spark dir: " + tachyonDir, t) + case e: Exception => + logError("Exception while deleting tachyon spark dir: " + tachyonDir, e) } } } diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala index c37e76f893605..d8ff4ff6bd42c 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -22,15 +22,10 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import tachyon.client.{WriteType, ReadType} +import tachyon.client.{ReadType, WriteType} import org.apache.spark.Logging import org.apache.spark.util.Utils -import org.apache.spark.serializer.Serializer - - -private class Entry(val size: Long) - /** * Stores BlockManager blocks on Tachyon. @@ -46,8 +41,8 @@ private class TachyonStore( tachyonManager.getFile(blockId.name).length } - override def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel): PutResult = { - putToTachyonStore(blockId, bytes, true) + override def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel): PutResult = { + putIntoTachyonStore(blockId, bytes, returnValues = true) } override def putValues( @@ -55,7 +50,7 @@ private class TachyonStore( values: ArrayBuffer[Any], level: StorageLevel, returnValues: Boolean): PutResult = { - return putValues(blockId, values.toIterator, level, returnValues) + putValues(blockId, values.toIterator, level, returnValues) } override def putValues( @@ -63,12 +58,12 @@ private class TachyonStore( values: Iterator[Any], level: StorageLevel, returnValues: Boolean): PutResult = { - logDebug("Attempting to write values for block " + blockId) - val _bytes = blockManager.dataSerialize(blockId, values) - putToTachyonStore(blockId, _bytes, returnValues) + logDebug(s"Attempting to write values for block $blockId") + val bytes = blockManager.dataSerialize(blockId, values) + putIntoTachyonStore(blockId, bytes, returnValues) } - private def putToTachyonStore( + private def putIntoTachyonStore( blockId: BlockId, bytes: ByteBuffer, returnValues: Boolean): PutResult = { @@ -76,7 +71,7 @@ private class TachyonStore( // duplicate does not copy buffer, so inexpensive val byteBuffer = bytes.duplicate() byteBuffer.rewind() - logDebug("Attempting to put block " + blockId + " into Tachyon") + 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) @@ -84,7 +79,7 @@ private class TachyonStore( 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))) + blockId, Utils.bytesToString(byteBuffer.limit), finishTime - startTime)) if (returnValues) { PutResult(bytes.limit(), Right(bytes.duplicate())) @@ -106,10 +101,9 @@ private class TachyonStore( 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) { + if (file == null || file.getLocationHosts.size == 0) { return None } val is = file.getInStream(ReadType.CACHE) @@ -121,16 +115,15 @@ private class TachyonStore( val fetchSize = is.read(bs, 0, size.asInstanceOf[Int]) buffer = ByteBuffer.wrap(bs) if (fetchSize != size) { - logWarning("Failed to fetch the block " + blockId + " from Tachyon : Size " + size + - " is not equal to fetched size " + fetchSize) + logWarning(s"Failed to fetch the block $blockId from Tachyon: Size $size " + + s"is not equal to fetched size $fetchSize") return None } } } catch { - case ioe: IOException => { - logWarning("Failed to fetch the block " + blockId + " from Tachyon", ioe) - return None - } + case ioe: IOException => + logWarning(s"Failed to fetch the block $blockId from Tachyon", ioe) + return None } Some(buffer) } diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index 226ed2a132b00..a107c5182b3be 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -22,7 +22,7 @@ import java.util.concurrent.ArrayBlockingQueue import akka.actor._ import util.Random -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.KryoSerializer @@ -48,7 +48,7 @@ private[spark] object ThreadingTest { val block = (1 to blockSize).map(_ => Random.nextInt()) val level = randomLevel() val startTime = System.currentTimeMillis() - manager.put(blockId, block.iterator, level, true) + manager.put(blockId, block.iterator, level, tellMaster = true) println("Pushed block " + blockId + " in " + (System.currentTimeMillis - startTime) + " ms") queue.add((blockId, block)) } @@ -101,7 +101,7 @@ private[spark] object ThreadingTest { conf) val blockManager = new BlockManager( "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf, - new SecurityManager(conf)) + new SecurityManager(conf), new MapOutputTrackerMaster(conf)) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue)) producers.foreach(_.start) 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 e1a1f209c9282..a2535e3c1c41f 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -22,6 +22,7 @@ import javax.servlet.DispatcherType import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import scala.annotation.tailrec +import scala.language.implicitConversions import scala.util.{Failure, Success, Try} import scala.xml.Node @@ -33,6 +34,7 @@ import org.json4s.JValue import org.json4s.jackson.JsonMethods.{pretty, render} import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.util.Utils /** * Utilities for launching a web server using Jetty's HTTP Server class @@ -104,10 +106,12 @@ private[spark] object JettyUtils extends Logging { def createRedirectHandler( srcPath: String, destPath: String, + beforeRedirect: HttpServletRequest => Unit = x => (), basePath: String = ""): ServletContextHandler = { val prefixedDestPath = attachPrefix(basePath, destPath) val servlet = new HttpServlet { override def doGet(request: HttpServletRequest, response: HttpServletResponse) { + beforeRedirect(request) // Make sure we don't end up with "//" in the middle val newUrl = new URL(new URL(request.getRequestURL.toString), prefixedDestPath).toString response.sendRedirect(newUrl) @@ -119,9 +123,10 @@ private[spark] object JettyUtils extends Logging { /** Create a handler for serving files from a static directory */ def createStaticHandler(resourceBase: String, path: String): ServletContextHandler = { val contextHandler = new ServletContextHandler + contextHandler.setInitParameter("org.eclipse.jetty.servlet.Default.gzip", "false") val staticHandler = new DefaultServlet val holder = new ServletHolder(staticHandler) - Option(getClass.getClassLoader.getResource(resourceBase)) match { + Option(Utils.getSparkClassLoader.getResource(resourceBase)) match { case Some(res) => holder.setInitParameter("resourceBase", res.toString) case None => @@ -132,11 +137,11 @@ private[spark] object JettyUtils extends Logging { contextHandler } - /** Add security filters, if any, do the given list of ServletContextHandlers */ - private def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { + /** Add filters, if any, to the given list of ServletContextHandlers */ + def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { val filters: Array[String] = conf.get("spark.ui.filters", "").split(',').map(_.trim()) filters.foreach { - case filter : String => + case filter : String => if (!filter.isEmpty) { logInfo("Adding filter: " + filter) val holder : FilterHolder = new FilterHolder() @@ -151,7 +156,7 @@ private[spark] object JettyUtils extends Logging { if (parts.length == 2) holder.setInitParameter(parts(0), parts(1)) } } - val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, + val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, DispatcherType.FORWARD, DispatcherType.INCLUDE, DispatcherType.REQUEST) handlers.foreach { case(handler) => handler.addFilter(holder, "/*", enumDispatcher) } } @@ -189,10 +194,16 @@ private[spark] object JettyUtils extends Logging { case s: Success[_] => (server, server.getConnectors.head.getLocalPort) case f: Failure[_] => + val nextPort = (currentPort + 1) % 65536 server.stop() - logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) - logInfo("Error was: " + f.toString) - connect((currentPort + 1) % 65536) + pool.stop() + val msg = s"Failed to create UI on port $currentPort. Trying again on port $nextPort." + if (f.toString.contains("Address already in use")) { + logWarning(s"$msg - $f") + } else { + logError(msg, f.exception) + } + connect(nextPort) } } 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 ef1ad872c8ef7..097a1b81e1dd1 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -17,106 +17,94 @@ package org.apache.spark.ui -import org.eclipse.jetty.servlet.ServletContextHandler - -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkEnv} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.scheduler._ import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.env.EnvironmentUI -import org.apache.spark.ui.exec.ExecutorsUI -import org.apache.spark.ui.jobs.JobProgressUI -import org.apache.spark.ui.storage.BlockManagerUI -import org.apache.spark.util.Utils +import org.apache.spark.ui.env.EnvironmentTab +import org.apache.spark.ui.exec.ExecutorsTab +import org.apache.spark.ui.jobs.JobProgressTab +import org.apache.spark.ui.storage.StorageTab -/** Top level user interface for Spark */ +/** + * Top level user interface for a Spark application. + */ private[spark] class SparkUI( val sc: SparkContext, - conf: SparkConf, + val conf: SparkConf, + val securityManager: SecurityManager, val listenerBus: SparkListenerBus, - val appName: String, + var appName: String, val basePath: String = "") - extends Logging { + extends WebUI(securityManager, SparkUI.getUIPort(conf), conf, basePath) + with Logging { - def this(sc: SparkContext) = this(sc, sc.conf, sc.listenerBus, sc.appName) + def this(sc: SparkContext) = this(sc, sc.conf, sc.env.securityManager, sc.listenerBus, sc.appName) def this(conf: SparkConf, listenerBus: SparkListenerBus, appName: String, basePath: String) = - this(null, conf, listenerBus, appName, basePath) + this(null, conf, new SecurityManager(conf), listenerBus, appName, basePath) + + def this( + conf: SparkConf, + securityManager: SecurityManager, + listenerBus: SparkListenerBus, + appName: String, + basePath: String) = + this(null, conf, securityManager, listenerBus, appName, basePath) // If SparkContext is not provided, assume the associated application is not live val live = sc != null - val securityManager = if (live) sc.env.securityManager else new SecurityManager(conf) - - private val bindHost = Utils.localHostName() - private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(bindHost) - private val port = conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt - private var serverInfo: Option[ServerInfo] = None - - private val storage = new BlockManagerUI(this) - private val jobs = new JobProgressUI(this) - private val env = new EnvironmentUI(this) - private val exec = new ExecutorsUI(this) - - val handlers: Seq[ServletContextHandler] = { - val metricsServletHandlers = if (live) { - SparkEnv.get.metricsSystem.getServletHandlers - } else { - Array[ServletContextHandler]() - } - storage.getHandlers ++ - jobs.getHandlers ++ - env.getHandlers ++ - exec.getHandlers ++ - metricsServletHandlers ++ - Seq[ServletContextHandler] ( - createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static"), - createRedirectHandler("/", "/stages", basePath) - ) - } - // Maintain executor storage status through Spark events val storageStatusListener = new StorageStatusListener - /** Bind the HTTP server which backs this web interface */ - def bind() { - try { - serverInfo = Some(startJettyServer(bindHost, port, handlers, sc.conf)) - logInfo("Started Spark Web UI at http://%s:%d".format(publicHost, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Spark JettyUtils", e) - System.exit(1) + initialize() + + /** Initialize all components of the server. */ + def initialize() { + listenerBus.addListener(storageStatusListener) + val jobProgressTab = new JobProgressTab(this) + attachTab(jobProgressTab) + attachTab(new StorageTab(this)) + attachTab(new EnvironmentTab(this)) + attachTab(new ExecutorsTab(this)) + attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) + attachHandler(createRedirectHandler("/", "/stages", basePath = basePath)) + attachHandler( + createRedirectHandler("/stages/stage/kill", "/stages", jobProgressTab.handleKillRequest)) + if (live) { + sc.env.metricsSystem.getServletHandlers.foreach(attachHandler) } } - def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) - - /** Initialize all components of the server */ - def start() { - storage.start() - jobs.start() - env.start() - exec.start() + /** Set the app name for this UI. */ + def setAppName(name: String) { + appName = name + } - // Storage status listener must receive events first, as other listeners depend on its state - listenerBus.addListener(storageStatusListener) - listenerBus.addListener(storage.listener) - listenerBus.addListener(jobs.listener) - listenerBus.addListener(env.listener) - listenerBus.addListener(exec.listener) + /** Register the given listener with the listener bus. */ + def registerListener(listener: SparkListener) { + listenerBus.addListener(listener) } - def stop() { - assert(serverInfo.isDefined, "Attempted to stop a SparkUI that was not bound to a server!") - serverInfo.get.server.stop() - logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) + /** Stop the server behind this web interface. Only valid after bind(). */ + override def stop() { + super.stop() + logInfo("Stopped Spark web UI at %s".format(appUIAddress)) } - private[spark] def appUIAddress = "http://" + publicHost + ":" + boundPort + /** + * Return the application UI host:port. This does not include the scheme (http://). + */ + private[spark] def appUIHostPort = publicHostName + ":" + boundPort + private[spark] def appUIAddress = s"http://$appUIHostPort" } private[spark] object SparkUI { - val DEFAULT_PORT = "4040" + val DEFAULT_PORT = 4040 val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" + + def getUIPort(conf: SparkConf): Int = { + conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) + } } 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 a487924effbff..1b104253d545d 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -17,16 +17,125 @@ package org.apache.spark.ui +import java.text.SimpleDateFormat +import java.util.{Locale, Date} + import scala.xml.Node +import org.apache.spark.Logging /** Utility functions for generating XML pages with spark content. */ -private[spark] object UIUtils { +private[spark] object UIUtils extends Logging { + + // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. + private val dateFormat = new ThreadLocal[SimpleDateFormat]() { + override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + } + + def formatDate(date: Date): String = dateFormat.get.format(date) + + def formatDate(timestamp: Long): String = dateFormat.get.format(new Date(timestamp)) + + def formatDuration(milliseconds: Long): String = { + if (milliseconds < 100) { + return "%d ms".format(milliseconds) + } + val seconds = milliseconds.toDouble / 1000 + if (seconds < 1) { + return "%.1f s".format(seconds) + } + if (seconds < 60) { + return "%.0f s".format(seconds) + } + val minutes = seconds / 60 + if (minutes < 10) { + return "%.1f min".format(minutes) + } else if (minutes < 60) { + return "%.0f min".format(minutes) + } + val hours = minutes / 60 + "%.1f h".format(hours) + } + + /** Generate a verbose human-readable string representing a duration such as "5 second 35 ms" */ + def formatDurationVerbose(ms: Long): String = { + try { + val second = 1000L + val minute = 60 * second + val hour = 60 * minute + val day = 24 * hour + val week = 7 * day + val year = 365 * day + + def toString(num: Long, unit: String): String = { + if (num == 0) { + "" + } else if (num == 1) { + s"$num $unit" + } else { + s"$num ${unit}s" + } + } + + val millisecondsString = if (ms >= second && ms % second == 0) "" else s"${ms % second} ms" + val secondString = toString((ms % minute) / second, "second") + val minuteString = toString((ms % hour) / minute, "minute") + val hourString = toString((ms % day) / hour, "hour") + val dayString = toString((ms % week) / day, "day") + val weekString = toString((ms % year) / week, "week") + val yearString = toString(ms / year, "year") + + Seq( + second -> millisecondsString, + minute -> s"$secondString $millisecondsString", + hour -> s"$minuteString $secondString", + day -> s"$hourString $minuteString $secondString", + week -> s"$dayString $hourString $minuteString", + year -> s"$weekString $dayString $hourString" + ).foreach { case (durationLimit, durationString) => + if (ms < durationLimit) { + // if time is less than the limit (upto year) + return durationString + } + } + // if time is more than a year + return s"$yearString $weekString $dayString" + } catch { + case e: Exception => + logError("Error converting time to string", e) + // if there is some error, return blank string + return "" + } + } - import Page._ + /** Generate a human-readable string representing a number (e.g. 100 K) */ + def formatNumber(records: Double): String = { + val trillion = 1e12 + val billion = 1e9 + val million = 1e6 + val thousand = 1e3 + + val (value, unit) = { + if (records >= 2*trillion) { + (records / trillion, " T") + } else if (records >= 2*billion) { + (records / billion, " B") + } else if (records >= 2*million) { + (records / million, " M") + } else if (records >= 2*thousand) { + (records / thousand, " K") + } else { + (records, "") + } + } + if (unit.isEmpty) { + "%d".formatLocal(Locale.US, value.toInt) + } else { + "%.1f%s".formatLocal(Locale.US, value, unit) + } + } // Yarn has to go through a proxy so the base uri is provided and has to be on all links - private[spark] val uiRoot : String = Option(System.getenv("APPLICATION_WEB_PROXY_BASE")). - getOrElse("") + val uiRoot : String = Option(System.getenv("APPLICATION_WEB_PROXY_BASE")).getOrElse("") def prependBaseUri(basePath: String = "", resource: String = "") = uiRoot + basePath + resource @@ -36,26 +145,14 @@ private[spark] object UIUtils { basePath: String, appName: String, title: String, - page: Page.Value) : Seq[Node] = { - val jobs = page match { - case Stages => -
  • Stages
  • - case _ =>
  • Stages
  • - } - val storage = page match { - case Storage => -
  • Storage
  • - case _ =>
  • Storage
  • - } - val environment = page match { - case Environment => -
  • Environment
  • - case _ =>
  • Environment
  • - } - val executors = page match { - case Executors => -
  • Executors
  • - case _ =>
  • Executors
  • + tabs: Seq[WebUITab], + activeTab: WebUITab, + refreshInterval: Option[Int] = None): Seq[Node] = { + + val header = tabs.map { tab => +
  • + {tab.name} +
  • } @@ -71,19 +168,13 @@ private[spark] object UIUtils { -
    @@ -129,21 +220,36 @@ private[spark] object UIUtils { /** Returns an HTML table constructed by generating a row for each object in a sequence. */ def listingTable[T]( headers: Seq[String], - makeRow: T => Seq[Node], - rows: Seq[T], + generateDataRow: T => Seq[Node], + data: Seq[T], fixedWidth: Boolean = false): Seq[Node] = { - val colWidth = 100.toDouble / headers.size - val colWidthAttr = if (fixedWidth) colWidth + "%" else "" var tableClass = "table table-bordered table-striped table-condensed sortable" if (fixedWidth) { tableClass += " table-fixed" } - + val colWidth = 100.toDouble / headers.size + val colWidthAttr = if (fixedWidth) colWidth + "%" else "" + val headerRow: Seq[Node] = { + // if none of the headers have "\n" in them + if (headers.forall(!_.contains("\n"))) { + // represent header as simple text + headers.map(h => {h}) + } else { + // represent header text as list while respecting "\n" + headers.map { case h => + +
      + { h.split("\n").map { case t =>
    • {t}
    • } } +
    + + } + } + } - {headers.map(h => )} + {headerRow} - {rows.map(r => makeRow(r))} + {data.map(r => generateDataRow(r))}
    {h}
    } 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 a7b872f3445a4..b08f308fda1dd 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -17,34 +17,134 @@ package org.apache.spark.ui -import java.text.SimpleDateFormat -import java.util.Date +import javax.servlet.http.HttpServletRequest + +import scala.collection.mutable.ArrayBuffer +import scala.xml.Node + +import org.eclipse.jetty.servlet.ServletContextHandler +import org.json4s.JsonAST.{JNothing, JValue} + +import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.util.Utils /** - * Utilities used throughout the web UI. + * The top level component of the UI hierarchy that contains the server. + * + * Each WebUI represents a collection of tabs, each of which in turn represents a collection of + * pages. The use of tabs is optional, however; a WebUI may choose to include pages directly. */ -private[spark] object WebUI { - // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. - private val dateFormat = new ThreadLocal[SimpleDateFormat]() { - override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") +private[spark] abstract class WebUI( + securityManager: SecurityManager, + port: Int, + conf: SparkConf, + basePath: String = "") + extends Logging { + + protected val tabs = ArrayBuffer[WebUITab]() + protected val handlers = ArrayBuffer[ServletContextHandler]() + protected var serverInfo: Option[ServerInfo] = None + protected val localHostName = Utils.localHostName() + protected val publicHostName = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHostName) + private val className = Utils.getFormattedClassName(this) + + def getTabs: Seq[WebUITab] = tabs.toSeq + def getHandlers: Seq[ServletContextHandler] = handlers.toSeq + + /** Attach a tab to this UI, along with all of its attached pages. */ + def attachTab(tab: WebUITab) { + tab.pages.foreach(attachPage) + tabs += tab } - def formatDate(date: Date): String = dateFormat.get.format(date) + /** Attach a page to this UI. */ + def attachPage(page: WebUIPage) { + val pagePath = "/" + page.prefix + attachHandler(createServletHandler(pagePath, + (request: HttpServletRequest) => page.render(request), securityManager, basePath)) + attachHandler(createServletHandler(pagePath.stripSuffix("/") + "/json", + (request: HttpServletRequest) => page.renderJson(request), securityManager, basePath)) + } - def formatDate(timestamp: Long): String = dateFormat.get.format(new Date(timestamp)) + /** Attach a handler to this UI. */ + def attachHandler(handler: ServletContextHandler) { + handlers += handler + serverInfo.foreach { info => + info.rootHandler.addHandler(handler) + if (!handler.isStarted) { + handler.start() + } + } + } - def formatDuration(milliseconds: Long): String = { - val seconds = milliseconds.toDouble / 1000 - if (seconds < 60) { - return "%.0f s".format(seconds) + /** Detach a handler from this UI. */ + def detachHandler(handler: ServletContextHandler) { + handlers -= handler + serverInfo.foreach { info => + info.rootHandler.removeHandler(handler) + if (handler.isStarted) { + handler.stop() + } } - val minutes = seconds / 60 - if (minutes < 10) { - return "%.1f min".format(minutes) - } else if (minutes < 60) { - return "%.0f min".format(minutes) + } + + /** Initialize all components of the server. */ + def initialize() + + /** Bind to the HTTP server behind this web interface. */ + def bind() { + assert(!serverInfo.isDefined, "Attempted to bind %s more than once!".format(className)) + try { + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf)) + logInfo("Started %s at http://%s:%d".format(className, publicHostName, boundPort)) + } catch { + case e: Exception => + logError("Failed to bind %s".format(className), e) + System.exit(1) } - val hours = minutes / 60 - return "%.1f h".format(hours) } + + /** Return the actual port to which this server is bound. Only valid after bind(). */ + def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) + + /** Stop the server behind this web interface. Only valid after bind(). */ + def stop() { + assert(serverInfo.isDefined, + "Attempted to stop %s before binding to a server!".format(className)) + serverInfo.get.server.stop() + } +} + + +/** + * A tab that represents a collection of pages. + * The prefix is appended to the parent address to form a full path, and must not contain slashes. + */ +private[spark] abstract class WebUITab(parent: WebUI, val prefix: String) { + val pages = ArrayBuffer[WebUIPage]() + val name = prefix.capitalize + + /** Attach a page to this tab. This prepends the page's prefix with the tab's own prefix. */ + def attachPage(page: WebUIPage) { + page.prefix = (prefix + "/" + page.prefix).stripSuffix("/") + pages += page + } + + /** Get a list of header tabs from the parent UI. */ + def headerTabs: Seq[WebUITab] = parent.getTabs +} + + +/** + * A page that represents the leaf node in the UI hierarchy. + * + * The direct parent of a WebUIPage is not specified as it can be either a WebUI or a WebUITab. + * If the parent is a WebUI, the prefix is appended to the parent's address to form a full path. + * Else, if the parent is a WebUITab, the prefix is appended to the super prefix of the parent + * to form a relative path. The prefix must not contain slashes. + */ +private[spark] abstract class WebUIPage(var prefix: String) { + def render(request: HttpServletRequest): Seq[Node] + def renderJson(request: HttpServletRequest): JValue = JNothing } diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala similarity index 62% rename from core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala rename to core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala index 23e90c34d5b33..b347eb1b83c1f 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala @@ -21,28 +21,12 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.eclipse.jetty.servlet.ServletContextHandler +import org.apache.spark.ui.{UIUtils, WebUIPage} -import org.apache.spark.scheduler._ -import org.apache.spark.ui._ -import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.Page.Environment - -private[ui] class EnvironmentUI(parent: SparkUI) { +private[ui] class EnvironmentPage(parent: EnvironmentTab) extends WebUIPage("") { private val appName = parent.appName private val basePath = parent.basePath - private var _listener: Option[EnvironmentListener] = None - - lazy val listener = _listener.get - - def start() { - _listener = Some(new EnvironmentListener) - } - - def getHandlers = Seq[ServletContextHandler]( - createServletHandler("/environment", - (request: HttpServletRequest) => render(request), parent.securityManager, basePath) - ) + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { val runtimeInformationTable = UIUtils.listingTable( @@ -61,7 +45,7 @@ private[ui] class EnvironmentUI(parent: SparkUI) {

    Classpath Entries

    {classpathEntriesTable} - UIUtils.headerSparkPage(content, basePath, appName, "Environment", Environment) + UIUtils.headerSparkPage(content, basePath, appName, "Environment", parent.headerTabs, parent) } private def propertyHeader = Seq("Name", "Value") @@ -70,23 +54,3 @@ private[ui] class EnvironmentUI(parent: SparkUI) { private def propertyRow(kv: (String, String)) = {kv._1}{kv._2} private def classPathRow(data: (String, String)) = {data._1}{data._2} } - -/** - * A SparkListener that prepares information to be displayed on the EnvironmentUI - */ -private[ui] class EnvironmentListener extends SparkListener { - var jvmInformation = Seq[(String, String)]() - var sparkProperties = Seq[(String, String)]() - var systemProperties = Seq[(String, String)]() - var classpathEntries = Seq[(String, String)]() - - override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { - synchronized { - val environmentDetails = environmentUpdate.environmentDetails - jvmInformation = environmentDetails("JVM Information") - sparkProperties = environmentDetails("Spark Properties") - systemProperties = environmentDetails("System Properties") - classpathEntries = environmentDetails("Classpath Entries") - } - } -} diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala new file mode 100644 index 0000000000000..bbbe55ecf44a1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.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.ui.env + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.scheduler._ +import org.apache.spark.ui._ + +private[ui] class EnvironmentTab(parent: SparkUI) extends WebUITab(parent, "environment") { + val appName = parent.appName + val basePath = parent.basePath + val listener = new EnvironmentListener + + attachPage(new EnvironmentPage(this)) + parent.registerListener(listener) +} + +/** + * :: DeveloperApi :: + * A SparkListener that prepares information to be displayed on the EnvironmentTab + */ +@DeveloperApi +class EnvironmentListener extends SparkListener { + var jvmInformation = Seq[(String, String)]() + var sparkProperties = Seq[(String, String)]() + var systemProperties = Seq[(String, String)]() + var classpathEntries = Seq[(String, String)]() + + override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { + synchronized { + val environmentDetails = environmentUpdate.environmentDetails + jvmInformation = environmentDetails("JVM Information") + sparkProperties = environmentDetails("Spark Properties") + systemProperties = environmentDetails("System Properties") + classpathEntries = environmentDetails("Classpath Entries") + } + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala similarity index 60% rename from core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala rename to core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 031ed88a493a8..6cb43c02b8f08 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -19,39 +19,20 @@ package org.apache.spark.ui.exec import javax.servlet.http.HttpServletRequest -import scala.collection.mutable.HashMap import scala.xml.Node -import org.eclipse.jetty.servlet.ServletContextHandler - -import org.apache.spark.ExceptionFailure -import org.apache.spark.scheduler._ -import org.apache.spark.storage.StorageStatusListener -import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.Page.Executors -import org.apache.spark.ui.{SparkUI, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[ui] class ExecutorsUI(parent: SparkUI) { +private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { private val appName = parent.appName private val basePath = parent.basePath - private var _listener: Option[ExecutorsListener] = None - - lazy val listener = _listener.get - - def start() { - _listener = Some(new ExecutorsListener(parent.storageStatusListener)) - } - - def getHandlers = Seq[ServletContextHandler]( - createServletHandler("/executors", - (request: HttpServletRequest) => render(request), parent.securityManager, basePath) - ) + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { val storageStatusList = listener.storageStatusList val maxMem = storageStatusList.map(_.maxMem).fold(0L)(_ + _) - val memUsed = storageStatusList.map(_.memUsed()).fold(0L)(_ + _) + val memUsed = storageStatusList.map(_.memUsed).fold(0L)(_ + _) val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).fold(0L)(_ + _) val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId) val execInfoSorted = execInfo.sortBy(_.getOrElse("Executor ID", "")) @@ -74,8 +55,8 @@ private[ui] class ExecutorsUI(parent: SparkUI) {
    ; - UIUtils.headerSparkPage( - content, basePath, appName, "Executors (" + execInfo.size + ")", Executors) + UIUtils.headerSparkPage(content, basePath, appName, "Executors (" + execInfo.size + ")", + parent.headerTabs, parent) } /** Header fields for the executors table */ @@ -125,9 +106,9 @@ private[ui] class ExecutorsUI(parent: SparkUI) { val execId = status.blockManagerId.executorId val hostPort = status.blockManagerId.hostPort val rddBlocks = status.blocks.size - val memUsed = status.memUsed() + val memUsed = status.memUsed val maxMem = status.maxMem - val diskUsed = status.diskUsed() + val diskUsed = status.diskUsed val activeTasks = listener.executorToTasksActive.getOrElse(execId, 0) val failedTasks = listener.executorToTasksFailed.getOrElse(execId, 0) val completedTasks = listener.executorToTasksComplete.getOrElse(execId, 0) @@ -158,55 +139,3 @@ private[ui] class ExecutorsUI(parent: SparkUI) { execFields.zip(execValues).toMap } } - -/** - * A SparkListener that prepares information to be displayed on the ExecutorsUI - */ -private[ui] class ExecutorsListener(storageStatusListener: StorageStatusListener) - extends SparkListener { - - val executorToTasksActive = HashMap[String, Int]() - val executorToTasksComplete = HashMap[String, Int]() - val executorToTasksFailed = HashMap[String, Int]() - val executorToDuration = HashMap[String, Long]() - val executorToShuffleRead = HashMap[String, Long]() - val executorToShuffleWrite = HashMap[String, Long]() - - def storageStatusList = storageStatusListener.storageStatusList - - override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { - val eid = formatExecutorId(taskStart.taskInfo.executorId) - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 - } - - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { - val info = taskEnd.taskInfo - if (info != null) { - val eid = formatExecutorId(info.executorId) - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 1) - 1 - executorToDuration(eid) = executorToDuration.getOrElse(eid, 0L) + info.duration - taskEnd.reason match { - case e: ExceptionFailure => - executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 - case _ => - executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 - } - - // Update shuffle read/write - val metrics = taskEnd.taskMetrics - if (metrics != null) { - metrics.shuffleReadMetrics.foreach { shuffleRead => - executorToShuffleRead(eid) = - executorToShuffleRead.getOrElse(eid, 0L) + shuffleRead.remoteBytesRead - } - metrics.shuffleWriteMetrics.foreach { shuffleWrite => - executorToShuffleWrite(eid) = - executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten - } - } - } - } - - // This addresses executor ID inconsistencies in the local mode - private def formatExecutorId(execId: String) = storageStatusListener.formatExecutorId(execId) -} 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 new file mode 100644 index 0000000000000..91d37b835b19d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.exec + +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.ui.{SparkUI, WebUITab} + +private[ui] class ExecutorsTab(parent: SparkUI) extends WebUITab(parent, "executors") { + val appName = parent.appName + val basePath = parent.basePath + val listener = new ExecutorsListener(parent.storageStatusListener) + + attachPage(new ExecutorsPage(this)) + parent.registerListener(listener) +} + +/** + * :: DeveloperApi :: + * A SparkListener that prepares information to be displayed on the ExecutorsTab + */ +@DeveloperApi +class ExecutorsListener(storageStatusListener: StorageStatusListener) + extends SparkListener { + + val executorToTasksActive = HashMap[String, Int]() + val executorToTasksComplete = HashMap[String, Int]() + val executorToTasksFailed = HashMap[String, Int]() + val executorToDuration = HashMap[String, Long]() + val executorToShuffleRead = HashMap[String, Long]() + val executorToShuffleWrite = HashMap[String, Long]() + + def storageStatusList = storageStatusListener.storageStatusList + + override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { + val eid = formatExecutorId(taskStart.taskInfo.executorId) + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 + } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + val info = taskEnd.taskInfo + if (info != null) { + val eid = formatExecutorId(info.executorId) + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 1) - 1 + executorToDuration(eid) = executorToDuration.getOrElse(eid, 0L) + info.duration + taskEnd.reason match { + case e: ExceptionFailure => + executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 + case _ => + executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 + } + + // Update shuffle read/write + val metrics = taskEnd.taskMetrics + if (metrics != null) { + metrics.shuffleReadMetrics.foreach { shuffleRead => + executorToShuffleRead(eid) = + executorToShuffleRead.getOrElse(eid, 0L) + shuffleRead.remoteBytesRead + } + metrics.shuffleWriteMetrics.foreach { shuffleWrite => + executorToShuffleWrite(eid) = + executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten + } + } + } + } + + // This addresses executor ID inconsistencies in the local mode + private def formatExecutorId(execId: String) = storageStatusListener.formatExecutorId(execId) +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala index 1dfe1d4f1fa11..2aaf6329b792d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala @@ -17,8 +17,14 @@ package org.apache.spark.ui.jobs -/** class for reporting aggregated metrics for each executors in stageUI */ -private[ui] class ExecutorSummary { +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * Class for reporting aggregated metrics for each executor in stage UI. + */ +@DeveloperApi +class ExecutorSummary { var taskTime : Long = 0 var failedTasks : Int = 0 var succeededTasks : Int = 0 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 73861ae6746da..c83e196c9c156 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 @@ -20,11 +20,12 @@ package org.apache.spark.ui.jobs import scala.collection.mutable import scala.xml.Node +import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing executor summary */ -private[ui] class ExecutorTable(stageId: Int, parent: JobProgressUI) { - private lazy val listener = parent.listener +private[ui] class ExecutorTable(stageId: Int, parent: JobProgressTab) { + private val listener = parent.listener def toNodeSeq: Seq[Node] = { listener.synchronized { @@ -69,7 +70,7 @@ private[ui] class ExecutorTable(stageId: Int, parent: JobProgressUI) { {k} {executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")} - {parent.formatDuration(v.taskTime)} + {UIUtils.formatDuration(v.taskTime)} {v.failedTasks + v.succeededTasks} {v.failedTasks} {v.succeededTasks} 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 cd4be57227a16..396cbcbc8d268 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 @@ -20,19 +20,22 @@ package org.apache.spark.ui.jobs import scala.collection.mutable.{HashMap, ListBuffer} import org.apache.spark.{ExceptionFailure, SparkConf, SparkContext, Success} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId /** + * :: DeveloperApi :: * Tracks task-level information to be displayed in the UI. * * All access to the data structures in this class must be synchronized on the * class, since the UI thread and the EventBus loop may otherwise be reading and * updating the internal data structures concurrently. */ -private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { +@DeveloperApi +class JobProgressListener(conf: SparkConf) extends SparkListener { import JobProgressListener._ @@ -74,8 +77,13 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { // Remove by stageId, rather than by StageInfo, in case the StageInfo is from storage poolToActiveStages(stageIdToPool(stageId)).remove(stageId) activeStages.remove(stageId) - completedStages += stage - trimIfNecessary(completedStages) + if (stage.failureReason.isEmpty) { + completedStages += stage + trimIfNecessary(completedStages) + } else { + failedStages += stage + trimIfNecessary(failedStages) + } } /** If stages is too large, remove and garbage collect old stages */ @@ -83,7 +91,6 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { if (stages.size > retainedStages) { val toRemove = math.max(retainedStages / 10, 1) stages.take(toRemove).foreach { s => - stageIdToTaskData.remove(s.stageId) stageIdToTime.remove(s.stageId) stageIdToShuffleRead.remove(s.stageId) stageIdToShuffleWrite.remove(s.stageId) @@ -92,8 +99,10 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { stageIdToTasksActive.remove(s.stageId) stageIdToTasksComplete.remove(s.stageId) stageIdToTasksFailed.remove(s.stageId) + stageIdToTaskData.remove(s.stageId) + stageIdToExecutorSummaries.remove(s.stageId) stageIdToPool.remove(s.stageId) - if (stageIdToDescription.contains(s.stageId)) {stageIdToDescription.remove(s.stageId)} + stageIdToDescription.remove(s.stageId) } stages.trimStart(toRemove) } @@ -214,28 +223,12 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { } } - override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized { - jobEnd.jobResult match { - case JobFailed(_, stageId) => - activeStages.get(stageId).foreach { s => - // Remove by stageId, rather than by StageInfo, in case the StageInfo is from storage - activeStages.remove(s.stageId) - poolToActiveStages(stageIdToPool(stageId)).remove(s.stageId) - failedStages += s - trimIfNecessary(failedStages) - } - case _ => - } - } - override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { synchronized { - val schedulingModeName = - environmentUpdate.environmentDetails("Spark Properties").toMap.get("spark.scheduler.mode") - schedulingMode = schedulingModeName match { - case Some(name) => Some(SchedulingMode.withName(name)) - case None => None - } + schedulingMode = environmentUpdate + .environmentDetails("Spark Properties").toMap + .get("spark.scheduler.mode") + .map(SchedulingMode.withName) } } @@ -256,7 +249,8 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { } -private[ui] case class TaskUIData( +@DeveloperApi +case class TaskUIData( taskInfo: TaskInfo, taskMetrics: Option[TaskMetrics] = None, exception: Option[ExceptionFailure] = None) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala similarity index 83% rename from core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala index 70d62b66a4829..0da62892118d4 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala @@ -22,16 +22,15 @@ import javax.servlet.http.HttpServletRequest import scala.xml.{Node, NodeSeq} import org.apache.spark.scheduler.Schedulable -import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing list of all ongoing and recently finished stages and pools */ -private[ui] class IndexPage(parent: JobProgressUI) { +private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") { private val appName = parent.appName private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc - private lazy val listener = parent.listener + private val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler def render(request: HttpServletRequest): Seq[Node] = { @@ -39,12 +38,15 @@ private[ui] class IndexPage(parent: JobProgressUI) { val activeStages = listener.activeStages.values.toSeq val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq - val now = System.currentTimeMillis() + val now = System.currentTimeMillis - val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) + val activeStagesTable = + new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, + parent, parent.killEnabled) val completedStagesTable = - new StageTable(completedStages.sortBy(_.submissionTime).reverse, parent) - val failedStagesTable = new StageTable(failedStages.sortBy(_.submissionTime).reverse, parent) + new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent) + val failedStagesTable = + new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent) // For now, pool information is only accessible in live UIs val pools = if (live) sc.getAllPools else Seq[Schedulable]() @@ -57,7 +59,7 @@ private[ui] class IndexPage(parent: JobProgressUI) { // Total duration is not meaningful unless the UI is live
  • Total Duration: - {parent.formatDuration(now - sc.startTime)} + {UIUtils.formatDuration(now - sc.startTime)}
  • }}
  • @@ -92,7 +94,7 @@ private[ui] class IndexPage(parent: JobProgressUI) {

    Failed Stages ({failedStages.size})

    ++ failedStagesTable.toNodeSeq - UIUtils.headerSparkPage(content, basePath, appName, "Spark Stages", Stages) + UIUtils.headerSparkPage(content, basePath, appName, "Spark Stages", parent.headerTabs, parent) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala new file mode 100644 index 0000000000000..3308c8c8a3d37 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.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.ui.jobs + +import javax.servlet.http.HttpServletRequest + +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.SchedulingMode +import org.apache.spark.ui.{SparkUI, WebUITab} + +/** Web UI showing progress status of all jobs in the given SparkContext. */ +private[ui] class JobProgressTab(parent: SparkUI) extends WebUITab(parent, "stages") { + val appName = parent.appName + val basePath = parent.basePath + val live = parent.live + val sc = parent.sc + val conf = if (live) sc.conf else new SparkConf + val killEnabled = conf.getBoolean("spark.ui.killEnabled", true) + val listener = new JobProgressListener(conf) + + attachPage(new JobProgressPage(this)) + attachPage(new StagePage(this)) + attachPage(new PoolPage(this)) + parent.registerListener(listener) + + def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) + + def handleKillRequest(request: HttpServletRequest) = { + if (killEnabled) { + 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)) { + sc.cancelStage(stageId) + } + // Do a quick pause here to give Spark time to kill the stage so it shows up as + // killed after the refresh. Note that this will block the serving thread so the + // time should be limited in duration. + Thread.sleep(100) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala deleted file mode 100644 index b2c67381cc3da..0000000000000 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ /dev/null @@ -1,60 +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.ui.jobs - -import javax.servlet.http.HttpServletRequest - -import org.eclipse.jetty.servlet.ServletContextHandler - -import org.apache.spark.SparkConf -import org.apache.spark.scheduler.SchedulingMode -import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.SparkUI -import org.apache.spark.util.Utils - -/** Web UI showing progress status of all jobs in the given SparkContext. */ -private[ui] class JobProgressUI(parent: SparkUI) { - val appName = parent.appName - val basePath = parent.basePath - val live = parent.live - val sc = parent.sc - - lazy val listener = _listener.get - lazy val isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) - - private val indexPage = new IndexPage(this) - private val stagePage = new StagePage(this) - private val poolPage = new PoolPage(this) - private var _listener: Option[JobProgressListener] = None - - def start() { - val conf = if (live) sc.conf else new SparkConf - _listener = Some(new JobProgressListener(conf)) - } - - def formatDuration(ms: Long) = Utils.msDurationToString(ms) - - def getHandlers = Seq[ServletContextHandler]( - createServletHandler("/stages/stage", - (request: HttpServletRequest) => stagePage.render(request), parent.securityManager, basePath), - createServletHandler("/stages/pool", - (request: HttpServletRequest) => poolPage.render(request), parent.securityManager, basePath), - createServletHandler("/stages", - (request: HttpServletRequest) => indexPage.render(request), parent.securityManager, basePath) - ) -} 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 bd33182b70059..0a2bf31833d2b 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 @@ -22,16 +22,15 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.scheduler.{Schedulable, StageInfo} -import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing specific pool details */ -private[ui] class PoolPage(parent: JobProgressUI) { +private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { private val appName = parent.appName private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc - private lazy val listener = parent.listener + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { @@ -41,7 +40,8 @@ private[ui] class PoolPage(parent: JobProgressUI) { case Some(s) => s.values.toSeq case None => Seq[StageInfo]() } - val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) + val activeStagesTable = + new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, parent) // For now, pool information is only accessible in live UIs val pools = if (live) Seq(sc.getPoolForName(poolName).get) else Seq[Schedulable]() @@ -51,8 +51,8 @@ private[ui] class PoolPage(parent: JobProgressUI) {

    Summary

    ++ poolTable.toNodeSeq ++

    {activeStages.size} Active Stages

    ++ activeStagesTable.toNodeSeq - UIUtils.headerSparkPage( - content, basePath, appName, "Fair Scheduler Pool: " + poolName, Stages) + UIUtils.headerSparkPage(content, basePath, appName, "Fair Scheduler Pool: " + poolName, + parent.headerTabs, parent) } } } 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 c5c8d8668740b..f4b68f241966d 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 @@ -24,10 +24,9 @@ import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.UIUtils /** Table showing list of pools */ -private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { +private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressTab) { private val basePath = parent.basePath - private val poolToActiveStages = listener.poolToActiveStages - private lazy val listener = parent.listener + private val listener = parent.listener def toNodeSeq: Seq[Node] = { listener.synchronized { @@ -48,7 +47,7 @@ private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { SchedulingMode - {rows.map(r => makeRow(r, poolToActiveStages))} + {rows.map(r => makeRow(r, listener.poolToActiveStages))} } 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 0c55f2ee7e944..4bce472036f7d 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 @@ -22,15 +22,14 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.ui.Page._ -import org.apache.spark.ui.{WebUI, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.{Utils, Distribution} /** Page showing statistics and task list for a given stage */ -private[ui] class StagePage(parent: JobProgressUI) { +private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { private val appName = parent.appName private val basePath = parent.basePath - private lazy val listener = parent.listener + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { @@ -42,8 +41,8 @@ private[ui] class StagePage(parent: JobProgressUI) {

    Summary Metrics

    No tasks have started yet

    Tasks

    No tasks have started yet
  • - return UIUtils.headerSparkPage( - content, basePath, appName, "Details for Stage %s".format(stageId), Stages) + return UIUtils.headerSparkPage(content, basePath, appName, + "Details for Stage %s".format(stageId), parent.headerTabs, parent) } val tasks = listener.stageIdToTaskData(stageId).values.toSeq.sortBy(_.taskInfo.launchTime) @@ -58,7 +57,7 @@ private[ui] class StagePage(parent: JobProgressUI) { val hasBytesSpilled = memoryBytesSpilled > 0 && diskBytesSpilled > 0 var activeTime = 0L - val now = System.currentTimeMillis() + val now = System.currentTimeMillis val tasksActive = listener.stageIdToTasksActive(stageId).values tasksActive.foreach(activeTime += _.timeRunning(now)) @@ -68,7 +67,7 @@ private[ui] class StagePage(parent: JobProgressUI) {
    • Total task time across all tasks: - {parent.formatDuration(listener.stageIdToTime.getOrElse(stageId, 0L) + activeTime)} + {UIUtils.formatDuration(listener.stageIdToTime.getOrElse(stageId, 0L) + activeTime)}
    • {if (hasShuffleRead)
    • @@ -119,13 +118,13 @@ private[ui] class StagePage(parent: JobProgressUI) { } val serializationQuantiles = "Result serialization time" +: Distribution(serializationTimes). - get.getQuantiles().map(ms => parent.formatDuration(ms.toLong)) + get.getQuantiles().map(ms => UIUtils.formatDuration(ms.toLong)) val serviceTimes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.executorRunTime.toDouble } val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles() - .map(ms => parent.formatDuration(ms.toLong)) + .map(ms => UIUtils.formatDuration(ms.toLong)) val gettingResultTimes = validTasks.map { case TaskUIData(info, _, _) => if (info.gettingResultTime > 0) { @@ -136,7 +135,7 @@ private[ui] class StagePage(parent: JobProgressUI) { } val gettingResultQuantiles = "Time spent fetching task results" +: Distribution(gettingResultTimes).get.getQuantiles().map { millis => - parent.formatDuration(millis.toLong) + UIUtils.formatDuration(millis.toLong) } // The scheduler delay includes the network delay to send the task to the worker // machine and to send back the result (but not the time to fetch the task result, @@ -153,7 +152,7 @@ private[ui] class StagePage(parent: JobProgressUI) { } val schedulerDelayQuantiles = "Scheduler delay" +: Distribution(schedulerDelays).get.getQuantiles().map { millis => - parent.formatDuration(millis.toLong) + UIUtils.formatDuration(millis.toLong) } def getQuantileCols(data: Seq[Double]) = @@ -204,8 +203,8 @@ private[ui] class StagePage(parent: JobProgressUI) {

      Aggregated Metrics by Executor

      ++ executorTable.toNodeSeq ++

      Tasks

      ++ taskTable - UIUtils.headerSparkPage( - content, basePath, appName, "Details for Stage %d".format(stageId), Stages) + UIUtils.headerSparkPage(content, basePath, appName, "Details for Stage %d".format(stageId), + parent.headerTabs, parent) } } @@ -217,8 +216,8 @@ private[ui] class StagePage(parent: JobProgressUI) { taskData match { case TaskUIData(info, metrics, exception) => val duration = if (info.status == "RUNNING") info.timeRunning(System.currentTimeMillis()) else metrics.map(_.executorRunTime).getOrElse(1L) - val formatDuration = if (info.status == "RUNNING") parent.formatDuration(duration) - else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("") + val formatDuration = if (info.status == "RUNNING") UIUtils.formatDuration(duration) + else metrics.map(m => UIUtils.formatDuration(m.executorRunTime)).getOrElse("") val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) @@ -233,8 +232,8 @@ private[ui] class StagePage(parent: JobProgressUI) { val maybeWriteTime = metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleWriteTime) val writeTimeSortable = maybeWriteTime.map(_.toString).getOrElse("") - val writeTimeReadable = maybeWriteTime.map( t => t / (1000 * 1000)).map { ms => - if (ms == 0) "" else parent.formatDuration(ms) + val writeTimeReadable = maybeWriteTime.map(t => t / (1000 * 1000)).map { ms => + if (ms == 0) "" else UIUtils.formatDuration(ms) }.getOrElse("") val maybeMemoryBytesSpilled = metrics.map(_.memoryBytesSpilled) @@ -252,15 +251,15 @@ private[ui] class StagePage(parent: JobProgressUI) { {info.status} {info.taskLocality} {info.host} - {WebUI.formatDate(new Date(info.launchTime))} + {UIUtils.formatDate(new Date(info.launchTime))} {formatDuration} - {if (gcTime > 0) parent.formatDuration(gcTime) else ""} + {if (gcTime > 0) UIUtils.formatDuration(gcTime) else ""} - {if (serializationTime > 0) parent.formatDuration(serializationTime) else ""} + {if (serializationTime > 0) UIUtils.formatDuration(serializationTime) else ""} {if (shuffleRead) { 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 ac61568af52d2..a3f824a4e1f57 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 @@ -23,34 +23,40 @@ import scala.collection.mutable.HashMap import scala.xml.Node import org.apache.spark.scheduler.{StageInfo, TaskInfo} -import org.apache.spark.ui.{WebUI, UIUtils} +import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ -private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { +private[ui] class StageTableBase( + stages: Seq[StageInfo], + parent: JobProgressTab, + killEnabled: Boolean = false) { + private val basePath = parent.basePath - private lazy val listener = parent.listener - private lazy val isFairScheduler = parent.isFairScheduler + private val listener = parent.listener + protected def isFairScheduler = parent.isFairScheduler + + protected def columns: Seq[Node] = { + Stage Id ++ + {if (isFairScheduler) {Pool Name} else Seq.empty} ++ + Description + Submitted + Duration + Tasks: Succeeded/Total + Shuffle Read + Shuffle Write + } def toNodeSeq: Seq[Node] = { listener.synchronized { - stageTable(stageRow, stages) + stageTable(renderStageRow, stages) } } /** Special table that merges two header cells. */ - private def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { + protected def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { - - - {if (isFairScheduler) {} else {}} - - - - - - - + {columns} {rows.map(r => makeRow(r))} @@ -71,24 +77,44 @@ private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { } - /** Render an HTML row that represents a stage */ - private def stageRow(s: StageInfo): Seq[Node] = { - val poolName = listener.stageIdToPool.get(s.stageId) + private def makeDescription(s: StageInfo): Seq[Node] = { + // scalastyle:off + val killLink = if (killEnabled) { + + (kill) + + } + // scalastyle:on + val nameLink = {s.name} - val description = listener.stageIdToDescription.get(s.stageId) - .map(d =>
      {d}
      {nameLink}
      ).getOrElse(nameLink) + + val details = if (s.details.nonEmpty) ( + + +show details + + + ) + + listener.stageIdToDescription.get(s.stageId) + .map(d =>
      {d}
      {nameLink} {killLink}
      ) + .getOrElse(
      {killLink} {nameLink} {details}
      ) + } + + protected def stageRow(s: StageInfo): Seq[Node] = { + val poolName = listener.stageIdToPool.get(s.stageId) val submissionTime = s.submissionTime match { - case Some(t) => WebUI.formatDate(new Date(t)) + case Some(t) => UIUtils.formatDate(new Date(t)) case None => "Unknown" } val finishTime = s.completionTime.getOrElse(System.currentTimeMillis) val duration = s.submissionTime.map { t => if (finishTime > t) finishTime - t else System.currentTimeMillis - t } - val formattedDuration = duration.map(d => parent.formatDuration(d)).getOrElse("Unknown") + val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") val startedTasks = listener.stageIdToTasksActive.getOrElse(s.stageId, HashMap[Long, TaskInfo]()).size val completedTasks = listener.stageIdToTasksComplete.getOrElse(s.stageId, 0) @@ -107,25 +133,42 @@ private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { case 0 => "" case b => Utils.bytesToString(b) } - - - - {if (isFairScheduler) { - - }} - - - - ++ + {if (isFairScheduler) { + - - - + } else { + Seq.empty + }} ++ + + + + + + + } + + /** Render an HTML row that represents a stage */ + private def renderStageRow(s: StageInfo): Seq[Node] = {stageRow(s)} +} + +private[ui] class FailedStageTable( + stages: Seq[StageInfo], + parent: JobProgressTab, + killEnabled: Boolean = false) + extends StageTableBase(stages, parent, killEnabled) { + + override protected def columns: Seq[Node] = super.columns ++ + + override protected def stageRow(s: StageInfo): Seq[Node] = { + val basicColumns = super.stageRow(s) + val failureReason = + basicColumns ++ failureReason } } 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 3f42eba4ece00..2155633b8096f 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 @@ -22,22 +22,22 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.storage.{BlockId, BlockStatus, StorageStatus, StorageUtils} -import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ -private[ui] class RDDPage(parent: BlockManagerUI) { +private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { private val appName = parent.appName private val basePath = parent.basePath - private lazy val listener = parent.listener + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { val rddId = request.getParameter("id").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(Seq[Node](), basePath, appName, "RDD Not Found", Storage) + return UIUtils.headerSparkPage(Seq[Node](), basePath, appName, "RDD Not Found", + parent.headerTabs, parent) } // Worker table @@ -95,8 +95,8 @@ private[ui] class RDDPage(parent: BlockManagerUI) { ; - UIUtils.headerSparkPage( - content, basePath, appName, "RDD Storage Info for " + rddInfo.name, Storage) + UIUtils.headerSparkPage(content, basePath, appName, "RDD Storage Info for " + rddInfo.name, + parent.headerTabs, parent) } /** Header fields for the worker table */ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala similarity index 91% rename from core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala rename to core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala index 0fa461e5e9d27..b66edd91f56c0 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala @@ -22,21 +22,19 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.storage.RDDInfo -import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ -private[ui] class IndexPage(parent: BlockManagerUI) { +private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { private val appName = parent.appName private val basePath = parent.basePath - private lazy val listener = parent.listener + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - val rdds = listener.rddInfoList val content = UIUtils.listingTable(rddHeader, rddRow, rdds) - UIUtils.headerSparkPage(content, basePath, appName, "Storage ", Storage) + UIUtils.headerSparkPage(content, basePath, appName, "Storage ", parent.headerTabs, parent) } /** Header fields for the RDD table */ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala similarity index 72% rename from core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala rename to core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index a7b24ff695214..c4bb7aab50393 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -17,44 +17,30 @@ package org.apache.spark.ui.storage -import javax.servlet.http.HttpServletRequest - import scala.collection.mutable -import org.eclipse.jetty.servlet.ServletContextHandler - +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ui._ -import org.apache.spark.ui.JettyUtils._ import org.apache.spark.scheduler._ import org.apache.spark.storage.{RDDInfo, StorageStatusListener, StorageUtils} /** Web UI showing storage status of all RDD's in the given SparkContext. */ -private[ui] class BlockManagerUI(parent: SparkUI) { +private[ui] class StorageTab(parent: SparkUI) extends WebUITab(parent, "storage") { val appName = parent.appName val basePath = parent.basePath + val listener = new StorageListener(parent.storageStatusListener) - private val indexPage = new IndexPage(this) - private val rddPage = new RDDPage(this) - private var _listener: Option[BlockManagerListener] = None - - lazy val listener = _listener.get - - def start() { - _listener = Some(new BlockManagerListener(parent.storageStatusListener)) - } - - def getHandlers = Seq[ServletContextHandler]( - createServletHandler("/storage/rdd", - (request: HttpServletRequest) => rddPage.render(request), parent.securityManager, basePath), - createServletHandler("/storage", - (request: HttpServletRequest) => indexPage.render(request), parent.securityManager, basePath) - ) + attachPage(new StoragePage(this)) + attachPage(new RDDPage(this)) + parent.registerListener(listener) } /** - * A SparkListener that prepares information to be displayed on the BlockManagerUI + * :: DeveloperApi :: + * A SparkListener that prepares information to be displayed on the BlockManagerUI. */ -private[ui] class BlockManagerListener(storageStatusListener: StorageStatusListener) +@DeveloperApi +class StorageListener(storageStatusListener: StorageStatusListener) extends SparkListener { private val _rddInfoMap = mutable.Map[Int, RDDInfo]() @@ -83,8 +69,8 @@ private[ui] class BlockManagerListener(storageStatusListener: StorageStatusListe } override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = synchronized { - val rddInfo = stageSubmitted.stageInfo.rddInfo - _rddInfoMap.getOrElseUpdate(rddInfo.id, rddInfo) + val rddInfos = stageSubmitted.stageInfo.rddInfos + rddInfos.foreach { info => _rddInfoMap.getOrElseUpdate(info.id, info) } } override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { 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 d0ff17db632c1..a8d12bb2a0165 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.util import scala.collection.JavaConversions.mapAsJavaMap import scala.concurrent.duration.{Duration, FiniteDuration} -import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem} +import akka.actor.{ActorSystem, ExtendedActorSystem} import com.typesafe.config.ConfigFactory import org.apache.log4j.{Level, Logger} @@ -41,7 +41,7 @@ private[spark] object AkkaUtils extends Logging { * If indestructible is set to true, the Actor System will continue running in the event * of a fatal exception. This is used by [[org.apache.spark.executor.Executor]]. */ - def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false, + def createActorSystem(name: String, host: String, port: Int, conf: SparkConf, securityManager: SecurityManager): (ActorSystem, Int) = { val akkaThreads = conf.getInt("spark.akka.threads", 4) @@ -101,12 +101,7 @@ private[spark] object AkkaUtils extends Logging { |akka.log-dead-letters-during-shutdown = $lifecycleEvents """.stripMargin)) - val actorSystem = if (indestructible) { - IndestructibleActorSystem(name, akkaConf) - } else { - ActorSystem(name, akkaConf) - } - + val actorSystem = ActorSystem(name, akkaConf) val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider val boundPort = provider.getDefaultAddress.port.get (actorSystem, boundPort) @@ -119,7 +114,7 @@ private[spark] object AkkaUtils extends Logging { /** Returns the default Spark timeout to use for Akka remote actor lookup. */ def lookupTimeout(conf: SparkConf): FiniteDuration = { - Duration.create(conf.get("spark.akka.lookupTimeout", "30").toLong, "seconds") + Duration.create(conf.getLong("spark.akka.lookupTimeout", 30), "seconds") } /** Returns the configured max frame size for Akka messages in bytes. */ diff --git a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala index c3692f2fd929b..1b2b1932e0c3d 100644 --- a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala @@ -28,7 +28,7 @@ import scala.collection.generic.Growable * class and modifies it such that only the top K elements are retained. * The top K elements are defined by an implicit Ordering[A]. */ -class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) +private[spark] class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) extends Iterable[A] with Growable[A] with Serializable { private val underlying = new JPriorityQueue[A](maxSize, ord) @@ -43,8 +43,11 @@ class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) } override def +=(elem: A): this.type = { - if (size < maxSize) underlying.offer(elem) - else maybeReplaceLowest(elem) + if (size < maxSize) { + underlying.offer(elem) + } else { + maybeReplaceLowest(elem) + } this } @@ -59,7 +62,8 @@ class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) if (head != null && ord.gt(a, head)) { underlying.poll() underlying.offer(a) - } else false + } else { + false + } } } - 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 cdbbc65292188..4916d9b86cca5 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.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 +import org.apache.spark.{Logging, SparkException} private[spark] object ClosureCleaner extends Logging { // Get an ASM class reader for a given class from the JAR that loaded it @@ -45,7 +45,7 @@ private[spark] object ClosureCleaner extends Logging { private def isClosure(cls: Class[_]): Boolean = { cls.getName.contains("$anonfun$") } - + // Get a list of the classes of the outer objects of a given closure object, obj; // the outer objects are defined as any closures that obj is nested within, plus // possibly the class that the outermost closure is in, if any. We stop searching @@ -63,7 +63,7 @@ private[spark] object ClosureCleaner extends Logging { } Nil } - + // Get a list of the outer objects for a given closure object. private def getOuterObjects(obj: AnyRef): List[AnyRef] = { for (f <- obj.getClass.getDeclaredFields if f.getName == "$outer") { @@ -76,7 +76,7 @@ private[spark] object ClosureCleaner extends Logging { } Nil } - + private def getInnerClasses(obj: AnyRef): List[Class[_]] = { val seen = Set[Class[_]](obj.getClass) var stack = List[Class[_]](obj.getClass) @@ -92,7 +92,7 @@ private[spark] object ClosureCleaner extends Logging { } return (seen - obj.getClass).toList } - + private def createNullValue(cls: Class[_]): AnyRef = { if (cls.isPrimitive) { new java.lang.Byte(0: Byte) // Should be convertible to any primitive type @@ -100,14 +100,17 @@ private[spark] object ClosureCleaner extends Logging { null } } - + def clean(func: AnyRef) { // TODO: cache outerClasses / innerClasses / accessedFields val outerClasses = getOuterClasses(func) val innerClasses = getInnerClasses(func) val outerObjects = getOuterObjects(func) - + val accessedFields = Map[Class[_], Set[String]]() + + getClassReader(func.getClass).accept(new ReturnStatementFinder(), 0) + for (cls <- outerClasses) accessedFields(cls) = Set[String]() for (cls <- func.getClass :: innerClasses) @@ -143,7 +146,7 @@ private[spark] object ClosureCleaner extends Logging { field.set(outer, value) } } - + if (outer != null) { // logInfo("2: Setting $outer on " + func.getClass + " to " + outer); val field = func.getClass.getDeclaredField("$outer") @@ -151,7 +154,7 @@ private[spark] object ClosureCleaner extends Logging { field.set(func, outer) } } - + private def instantiateClass(cls: Class[_], outer: AnyRef, inInterpreter: Boolean): AnyRef = { // logInfo("Creating a " + cls + " with outer = " + outer) if (!inInterpreter) { @@ -180,6 +183,24 @@ private[spark] object ClosureCleaner extends Logging { } } +private[spark] +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") + } + } + } + } else { + new MethodVisitor(ASM4) {} + } + } +} + private[spark] class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor(ASM4) { override def visitMethod(access: Int, name: String, desc: String, @@ -192,7 +213,7 @@ class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor } } } - + 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. @@ -209,12 +230,12 @@ class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor private[spark] class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM4) { var myName: String = null - + override def visit(version: Int, access: Int, name: String, sig: String, superName: String, interfaces: Array[String]) { myName = name } - + override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { new MethodVisitor(ASM4) { diff --git a/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala b/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala index db3db87e6618e..e4c254b9dd6b9 100644 --- a/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala +++ b/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala @@ -22,8 +22,8 @@ import java.util import scala.Array import scala.reflect._ -object CollectionsUtils { - def makeBinarySearch[K <% Ordered[K] : ClassTag] : (Array[K], K) => Int = { +private[spark] object CollectionsUtils { + def makeBinarySearch[K : Ordering : ClassTag] : (Array[K], K) => Int = { classTag[K] match { case ClassTag.Float => (l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[Float]], x.asInstanceOf[Float]) 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 5b347555fe708..a465298c8c5ab 100644 --- a/core/src/main/scala/org/apache/spark/util/Distribution.scala +++ b/core/src/main/scala/org/apache/spark/util/Distribution.scala @@ -29,7 +29,7 @@ import scala.collection.immutable.IndexedSeq * * Assumes you are giving it a non-empty set of data */ -class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) { +private[spark] class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) { require(startIdx < endIdx) def this(data: Traversable[Double]) = this(data.toArray, 0, data.size) java.util.Arrays.sort(data, startIdx, endIdx) @@ -69,7 +69,7 @@ class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) } } -object Distribution { +private[spark] object Distribution { def apply(data: Traversable[Double]): Option[Distribution] = { if (data.size > 0) { diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index a0c07e32fdc98..6a95dc06e155d 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -17,15 +17,17 @@ package org.apache.spark.util -import java.io._ +import java.io.{BufferedOutputStream, FileOutputStream, IOException, PrintWriter} import java.net.URI import java.text.SimpleDateFormat import java.util.Date -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream -import org.apache.hadoop.fs.{FSDataOutputStream, Path} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} +import org.apache.hadoop.fs.permission.FsPermission import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec /** @@ -36,12 +38,14 @@ import org.apache.spark.io.CompressionCodec * @param compress Whether to compress output * @param overwrite Whether to overwrite existing files */ -class FileLogger( +private[spark] class FileLogger( logDir: String, - conf: SparkConf = new SparkConf, + sparkConf: SparkConf, + hadoopConf: Configuration = SparkHadoopUtil.get.newConfiguration(), outputBufferSize: Int = 8 * 1024, // 8 KB compress: Boolean = false, - overwrite: Boolean = true) + overwrite: Boolean = true, + dirPermissions: Option[FsPermission] = None) extends Logging { private val dateFormat = new ThreadLocal[SimpleDateFormat]() { @@ -49,17 +53,29 @@ class FileLogger( } private val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) - private var fileIndex = 0 + var fileIndex = 0 // Only used if compression is enabled - private lazy val compressionCodec = CompressionCodec.createCodec(conf) + private lazy val compressionCodec = CompressionCodec.createCodec(sparkConf) // Only defined if the file system scheme is not local private var hadoopDataStream: Option[FSDataOutputStream] = None - private var writer: Option[PrintWriter] = { + // The Hadoop APIs have changed over time, so we use reflection to figure out + // the correct method to use to flush a hadoop data stream. See SPARK-1518 + // for details. + private val hadoopFlushMethod = { + val cls = classOf[FSDataOutputStream] + scala.util.Try(cls.getMethod("hflush")).getOrElse(cls.getMethod("sync")) + } + + private var writer: Option[PrintWriter] = None + + /** + * Start this logger by creating the logging directory. + */ + def start() { createLogDir() - Some(createWriter()) } /** @@ -79,29 +95,39 @@ class FileLogger( if (!fileSystem.mkdirs(path)) { throw new IOException("Error in creating log directory: %s".format(logDir)) } + if (dirPermissions.isDefined) { + val fsStatus = fileSystem.getFileStatus(path) + if (fsStatus.getPermission.toShort != dirPermissions.get.toShort) { + fileSystem.setPermission(path, dirPermissions.get) + } + } } /** * Create a new writer for the file identified by the given path. + * If the permissions are not passed in, it will default to use the permissions + * (dirPermissions) used when class was instantiated. */ - private def createWriter(): PrintWriter = { - val logPath = logDir + "/" + fileIndex + private def createWriter(fileName: String, perms: Option[FsPermission] = None): PrintWriter = { + val logPath = logDir + "/" + fileName val uri = new URI(logPath) + val path = new Path(logPath) + val defaultFs = FileSystem.getDefaultUri(hadoopConf).getScheme + val isDefaultLocal = defaultFs == null || defaultFs == "file" /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). * Therefore, for local files, use FileOutputStream instead. */ - val dstream = uri.getScheme match { - case "file" | null => + val dstream = + if ((isDefaultLocal && uri.getScheme == null) || uri.getScheme == "file") { // Second parameter is whether to append - new FileOutputStream(logPath, !overwrite) - - case _ => - val path = new Path(logPath) + new FileOutputStream(uri.getPath, !overwrite) + } else { hadoopDataStream = Some(fileSystem.create(path, overwrite)) hadoopDataStream.get - } + } - val bstream = new FastBufferedOutputStream(dstream, outputBufferSize) + perms.orElse(dirPermissions).foreach { p => fileSystem.setPermission(path, p) } + val bstream = new BufferedOutputStream(dstream, outputBufferSize) val cstream = if (compress) compressionCodec.compressedOutputStream(bstream) else bstream new PrintWriter(cstream) } @@ -112,8 +138,10 @@ class FileLogger( * @param withTime Whether to prepend message with a timestamp */ def log(msg: String, withTime: Boolean = false) { - val writeInfo = if (!withTime) msg else { - val date = new Date(System.currentTimeMillis()) + val writeInfo = if (!withTime) { + msg + } else { + val date = new Date(System.currentTimeMillis) dateFormat.get.format(date) + ": " + msg } writer.foreach(_.print(writeInfo)) @@ -129,13 +157,13 @@ class FileLogger( /** * Flush the writer to disk manually. * - * If the Hadoop FileSystem is used, the underlying FSDataOutputStream (r1.0.4) must be - * sync()'ed manually as it does not support flush(), which is invoked by when higher - * level streams are flushed. + * When using a Hadoop filesystem, we need to invoke the hflush or sync + * method. In HDFS, hflush guarantees that the data gets to all the + * DataNodes. */ def flush() { writer.foreach(_.flush()) - hadoopDataStream.foreach(_.sync()) + hadoopDataStream.foreach(hadoopFlushMethod.invoke(_)) } /** @@ -147,13 +175,18 @@ class FileLogger( } /** - * Start a writer for a new file if one does not already exit. + * Start a writer for a new file, closing the existing one if it exists. + * @param fileName Name of the new file, defaulting to the file index if not provided. + * @param perms Permissions to put on the new file. */ - def start() { - writer.getOrElse { - fileIndex += 1 - writer = Some(createWriter()) + def newFile(fileName: String = "", perms: Option[FsPermission] = None) { + fileIndex += 1 + writer.foreach(_.close()) + val name = fileName match { + case "" => fileIndex.toString + case _ => fileName } + writer = Some(createWriter(name, perms)) } /** diff --git a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala deleted file mode 100644 index 4188a869c13da..0000000000000 --- a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala +++ /dev/null @@ -1,68 +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. - */ - -// Must be in akka.actor package as ActorSystemImpl is protected[akka]. -package akka.actor - -import scala.util.control.{ControlThrowable, NonFatal} - -import com.typesafe.config.Config - -/** - * An akka.actor.ActorSystem which refuses to shut down in the event of a fatal exception - * This is necessary as Spark Executors are allowed to recover from fatal exceptions - * (see org.apache.spark.executor.Executor) - */ -object IndestructibleActorSystem { - def apply(name: String, config: Config): ActorSystem = - apply(name, config, ActorSystem.findClassLoader()) - - def apply(name: String, config: Config, classLoader: ClassLoader): ActorSystem = - new IndestructibleActorSystemImpl(name, config, classLoader).start() -} - -private[akka] class IndestructibleActorSystemImpl( - override val name: String, - applicationConfig: Config, - classLoader: ClassLoader) - extends ActorSystemImpl(name, applicationConfig, classLoader) { - - protected override def uncaughtExceptionHandler: Thread.UncaughtExceptionHandler = { - val fallbackHandler = super.uncaughtExceptionHandler - - new Thread.UncaughtExceptionHandler() { - def uncaughtException(thread: Thread, cause: Throwable): Unit = { - if (isFatalError(cause) && !settings.JvmExitOnFatalError) { - log.error(cause, "Uncaught fatal error from thread [{}] not shutting down " + - "ActorSystem [{}] tolerating and continuing.... ", thread.getName, name) - // shutdown() //TODO make it configurable - } else { - fallbackHandler.uncaughtException(thread, cause) - } - } - } - } - - def isFatalError(e: Throwable): Boolean = { - e match { - case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable => - false - case _ => - true - } - } -} 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 2155a8888c85c..7cecbfe62a382 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -62,6 +62,10 @@ private[spark] object JsonProtocol { blockManagerRemovedToJson(blockManagerRemoved) case unpersistRDD: SparkListenerUnpersistRDD => unpersistRDDToJson(unpersistRDD) + case applicationStart: SparkListenerApplicationStart => + applicationStartToJson(applicationStart) + case applicationEnd: SparkListenerApplicationEnd => + applicationEndToJson(applicationEnd) // Not used, but keeps compiler happy case SparkListenerShutdown => JNothing @@ -84,30 +88,27 @@ private[spark] object JsonProtocol { def taskStartToJson(taskStart: SparkListenerTaskStart): JValue = { val taskInfo = taskStart.taskInfo - val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing ("Event" -> Utils.getFormattedClassName(taskStart)) ~ ("Stage ID" -> taskStart.stageId) ~ - ("Task Info" -> taskInfoJson) + ("Task Info" -> taskInfoToJson(taskInfo)) } def taskGettingResultToJson(taskGettingResult: SparkListenerTaskGettingResult): JValue = { val taskInfo = taskGettingResult.taskInfo - val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing ("Event" -> Utils.getFormattedClassName(taskGettingResult)) ~ - ("Task Info" -> taskInfoJson) + ("Task Info" -> taskInfoToJson(taskInfo)) } def taskEndToJson(taskEnd: SparkListenerTaskEnd): JValue = { val taskEndReason = taskEndReasonToJson(taskEnd.reason) val taskInfo = taskEnd.taskInfo - val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing val taskMetrics = taskEnd.taskMetrics val taskMetricsJson = if (taskMetrics != null) taskMetricsToJson(taskMetrics) else JNothing ("Event" -> Utils.getFormattedClassName(taskEnd)) ~ ("Stage ID" -> taskEnd.stageId) ~ ("Task Type" -> taskEnd.taskType) ~ ("Task End Reason" -> taskEndReason) ~ - ("Task Info" -> taskInfoJson) ~ + ("Task Info" -> taskInfoToJson(taskInfo)) ~ ("Task Metrics" -> taskMetricsJson) } @@ -157,21 +158,36 @@ private[spark] object JsonProtocol { ("RDD ID" -> unpersistRDD.rddId) } + def applicationStartToJson(applicationStart: SparkListenerApplicationStart): JValue = { + ("Event" -> Utils.getFormattedClassName(applicationStart)) ~ + ("App Name" -> applicationStart.appName) ~ + ("Timestamp" -> applicationStart.time) ~ + ("User" -> applicationStart.sparkUser) + } + + def applicationEndToJson(applicationEnd: SparkListenerApplicationEnd): JValue = { + ("Event" -> Utils.getFormattedClassName(applicationEnd)) ~ + ("Timestamp" -> applicationEnd.time) + } + /** ------------------------------------------------------------------- * * JSON serialization methods for classes SparkListenerEvents depend on | * -------------------------------------------------------------------- */ def stageInfoToJson(stageInfo: StageInfo): JValue = { - val rddInfo = rddInfoToJson(stageInfo.rddInfo) + val rddInfo = JArray(stageInfo.rddInfos.map(rddInfoToJson).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) ("Stage ID" -> stageInfo.stageId) ~ ("Stage Name" -> stageInfo.name) ~ ("Number of Tasks" -> stageInfo.numTasks) ~ ("RDD Info" -> rddInfo) ~ + ("Details" -> stageInfo.details) ~ ("Submission Time" -> submissionTime) ~ ("Completion Time" -> completionTime) ~ + ("Failure Reason" -> failureReason) ~ ("Emitted Task Size Warning" -> stageInfo.emittedTaskSizeWarning) } @@ -193,7 +209,8 @@ private[spark] object JsonProtocol { taskMetrics.shuffleReadMetrics.map(shuffleReadMetricsToJson).getOrElse(JNothing) val shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics.map(shuffleWriteMetricsToJson).getOrElse(JNothing) - val updatedBlocks = taskMetrics.updatedBlocks.map { blocks => + val updatedBlocks = + taskMetrics.updatedBlocks.map { blocks => JArray(blocks.toList.map { case (id, status) => ("Block ID" -> id.toString) ~ ("Status" -> blockStatusToJson(status)) @@ -259,9 +276,7 @@ private[spark] object JsonProtocol { val json = jobResult match { case JobSucceeded => Utils.emptyJson case jobFailed: JobFailed => - val exception = exceptionToJson(jobFailed.exception) - ("Exception" -> exception) ~ - ("Failed Stage ID" -> jobFailed.failedStageId) + JObject("Exception" -> exceptionToJson(jobFailed.exception)) } ("Result" -> result) ~ json } @@ -346,6 +361,8 @@ private[spark] object JsonProtocol { val blockManagerAdded = Utils.getFormattedClassName(SparkListenerBlockManagerAdded) val blockManagerRemoved = Utils.getFormattedClassName(SparkListenerBlockManagerRemoved) val unpersistRDD = Utils.getFormattedClassName(SparkListenerUnpersistRDD) + val applicationStart = Utils.getFormattedClassName(SparkListenerApplicationStart) + val applicationEnd = Utils.getFormattedClassName(SparkListenerApplicationEnd) (json \ "Event").extract[String] match { case `stageSubmitted` => stageSubmittedFromJson(json) @@ -359,6 +376,8 @@ private[spark] object JsonProtocol { case `blockManagerAdded` => blockManagerAddedFromJson(json) case `blockManagerRemoved` => blockManagerRemovedFromJson(json) case `unpersistRDD` => unpersistRDDFromJson(json) + case `applicationStart` => applicationStartFromJson(json) + case `applicationEnd` => applicationEndFromJson(json) } } @@ -430,6 +449,17 @@ private[spark] object JsonProtocol { SparkListenerUnpersistRDD((json \ "RDD ID").extract[Int]) } + def applicationStartFromJson(json: JValue): SparkListenerApplicationStart = { + val appName = (json \ "App Name").extract[String] + val time = (json \ "Timestamp").extract[Long] + val sparkUser = (json \ "User").extract[String] + SparkListenerApplicationStart(appName, time, sparkUser) + } + + def applicationEndFromJson(json: JValue): SparkListenerApplicationEnd = { + SparkListenerApplicationEnd((json \ "Timestamp").extract[Long]) + } + /** --------------------------------------------------------------------- * * JSON deserialization methods for classes SparkListenerEvents depend on | @@ -439,14 +469,17 @@ private[spark] object JsonProtocol { val stageId = (json \ "Stage ID").extract[Int] val stageName = (json \ "Stage Name").extract[String] val numTasks = (json \ "Number of Tasks").extract[Int] - val rddInfo = rddInfoFromJson(json \ "RDD Info") + val rddInfos = (json \ "RDD Info").extract[List[JValue]].map(rddInfoFromJson) + 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]) + val failureReason = Utils.jsonOption(json \ "Failure Reason").map(_.extract[String]) val emittedTaskSizeWarning = (json \ "Emitted Task Size Warning").extract[Boolean] - val stageInfo = new StageInfo(stageId, stageName, numTasks, rddInfo) + val stageInfo = new StageInfo(stageId, stageName, numTasks, rddInfos, details) stageInfo.submissionTime = submissionTime stageInfo.completionTime = completionTime + stageInfo.failureReason = failureReason stageInfo.emittedTaskSizeWarning = emittedTaskSizeWarning stageInfo } @@ -472,6 +505,9 @@ private[spark] object JsonProtocol { } def taskMetricsFromJson(json: JValue): TaskMetrics = { + if (json == JNothing) { + return TaskMetrics.empty + } val metrics = new TaskMetrics metrics.hostname = (json \ "Host Name").extract[String] metrics.executorDeserializeTime = (json \ "Executor Deserialize Time").extract[Long] @@ -485,13 +521,14 @@ private[spark] object JsonProtocol { Utils.jsonOption(json \ "Shuffle Read Metrics").map(shuffleReadMetricsFromJson) metrics.shuffleWriteMetrics = Utils.jsonOption(json \ "Shuffle Write Metrics").map(shuffleWriteMetricsFromJson) - metrics.updatedBlocks = Utils.jsonOption(json \ "Updated Blocks").map { value => - value.extract[List[JValue]].map { block => - val id = BlockId((block \ "Block ID").extract[String]) - val status = blockStatusFromJson(block \ "Status") - (id, status) + metrics.updatedBlocks = + Utils.jsonOption(json \ "Updated Blocks").map { value => + value.extract[List[JValue]].map { block => + val id = BlockId((block \ "Block ID").extract[String]) + val status = blockStatusFromJson(block \ "Status") + (id, status) + } } - } metrics } @@ -561,8 +598,7 @@ private[spark] object JsonProtocol { case `jobSucceeded` => JobSucceeded case `jobFailed` => val exception = exceptionFromJson(json \ "Exception") - val failedStageId = (json \ "Failed Stage ID").extract[Int] - new JobFailed(exception, failedStageId) + new JobFailed(exception) } } @@ -579,7 +615,7 @@ private[spark] object JsonProtocol { val rddInfo = new RDDInfo(rddId, name, numPartitions, storageLevel) rddInfo.numCachedPartitions = numCachedPartitions rddInfo.memSize = memSize - rddInfo.tachyonSize = tachyonSize + rddInfo.tachyonSize = tachyonSize rddInfo.diskSize = diskSize rddInfo } @@ -612,11 +648,11 @@ private[spark] object JsonProtocol { } def propertiesFromJson(json: JValue): Properties = { - val properties = new Properties() - if (json != JNothing) { - mapFromJson(json).map { case (k, v) => properties.setProperty(k, v) } - } - properties + Utils.jsonOption(json).map { value => + val properties = new Properties + mapFromJson(json).foreach { case (k, v) => properties.setProperty(k, v) } + properties + }.getOrElse(null) } def UUIDFromJson(json: JValue): UUID = { 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 0448919e09161..2889e171f627e 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -62,8 +62,8 @@ private[spark] class MetadataCleaner( private[spark] object MetadataCleanerType extends Enumeration { - val MAP_OUTPUT_TRACKER, SPARK_CONTEXT, HTTP_BROADCAST, DAG_SCHEDULER, RESULT_TASK, - SHUFFLE_MAP_TASK, BLOCK_MANAGER, SHUFFLE_BLOCK_MANAGER, BROADCAST_VARS = Value + val MAP_OUTPUT_TRACKER, SPARK_CONTEXT, HTTP_BROADCAST, BLOCK_MANAGER, + SHUFFLE_BLOCK_MANAGER, BROADCAST_VARS = Value type MetadataCleanerType = Value @@ -78,20 +78,26 @@ private[spark] object MetadataCleaner { conf.getInt("spark.cleaner.ttl", -1) } - def getDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType): Int = - { - conf.get(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds(conf).toString) - .toInt + def getDelaySeconds( + conf: SparkConf, + cleanerType: MetadataCleanerType.MetadataCleanerType): Int = { + conf.get(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds(conf).toString).toInt } - def setDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType, - delay: Int) - { + def setDelaySeconds( + conf: SparkConf, + cleanerType: MetadataCleanerType.MetadataCleanerType, + delay: Int) { conf.set(MetadataCleanerType.systemProperty(cleanerType), delay.toString) } + /** + * Set the default delay time (in seconds). + * @param conf SparkConf instance + * @param delay default delay time to set + * @param resetAll whether to reset all to default + */ def setDelaySeconds(conf: SparkConf, delay: Int, resetAll: Boolean = true) { - // override for all ? conf.set("spark.cleaner.ttl", delay.toString) if (resetAll) { for (cleanerType <- MetadataCleanerType.values) { 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 a6b39247a54ca..74fa77b68de0b 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -17,13 +17,17 @@ package org.apache.spark.util +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A tuple of 2 elements. This can be used as an alternative to Scala's Tuple2 when we want to * minimize object allocation. * * @param _1 Element 1 of this MutablePair * @param _2 Element 2 of this MutablePair */ +@DeveloperApi case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T1, @specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T2] (var _1: T1, var _2: T2) diff --git a/core/src/main/scala/org/apache/spark/util/NextIterator.scala b/core/src/main/scala/org/apache/spark/util/NextIterator.scala index 8266e5e495efc..e5c732a5a559b 100644 --- a/core/src/main/scala/org/apache/spark/util/NextIterator.scala +++ b/core/src/main/scala/org/apache/spark/util/NextIterator.scala @@ -19,7 +19,7 @@ package org.apache.spark.util /** Provides a basic/boilerplate Iterator implementation. */ private[spark] abstract class NextIterator[U] extends Iterator[U] { - + private var gotNext = false private var nextValue: U = _ private var closed = false @@ -34,7 +34,7 @@ private[spark] abstract class NextIterator[U] extends Iterator[U] { * This convention is required because `null` may be a valid value, * and using `Option` seems like it might create unnecessary Some/None * instances, given some iterators might be called in a tight loop. - * + * * @return U, or set 'finished' when done */ protected def getNext(): U diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/VectorRDDs.scala b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala similarity index 70% rename from mllib/src/main/scala/org/apache/spark/mllib/rdd/VectorRDDs.scala rename to core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala index 9096d6a1a16d6..3abc12681fe9a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/VectorRDDs.scala +++ b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala @@ -15,18 +15,18 @@ * limitations under the License. */ -package org.apache.spark.mllib.rdd - -import org.apache.spark.rdd.RDD -import org.apache.spark.mllib.linalg.{Vectors, Vector} +package org.apache.spark.util /** - * Factory methods for `RDD[Vector]`. + * A class loader which makes findClass accesible to the child */ -object VectorRDDs { +private[spark] class ParentClassLoader(parent: ClassLoader) extends ClassLoader(parent) { + + override def findClass(name: String) = { + super.findClass(name) + } - /** - * Converts an `RDD[Array[Double]]` to `RDD[Vector]`. - */ - def fromArrayRDD(rdd: RDD[Array[Double]]): RDD[Vector] = rdd.map(v => Vectors.dense(v)) + override def loadClass(name: String): Class[_] = { + super.loadClass(name) + } } diff --git a/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala b/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala deleted file mode 100644 index 21a88eea3bbc2..0000000000000 --- a/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala +++ /dev/null @@ -1,52 +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.util - -import java.io.{Externalizable, ObjectInput, ObjectOutput} - -import com.clearspring.analytics.stream.cardinality.{HyperLogLog, ICardinality} - -/** - * A wrapper around [[com.clearspring.analytics.stream.cardinality.HyperLogLog]] that is - * serializable. - */ -private[spark] -class SerializableHyperLogLog(var value: ICardinality) extends Externalizable { - - def this() = this(null) // For deserialization - - def merge(other: SerializableHyperLogLog) = new SerializableHyperLogLog(value.merge(other.value)) - - def add[T](elem: T) = { - this.value.offer(elem) - this - } - - def readExternal(in: ObjectInput) { - val byteLength = in.readInt() - val bytes = new Array[Byte](byteLength) - in.readFully(bytes) - value = HyperLogLog.Builder.build(bytes) - } - - def writeExternal(out: ObjectOutput) { - val bytes = value.getBytes() - out.writeInt(bytes.length) - out.write(bytes) - } -} 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 b955612ca7749..08465575309c6 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -27,9 +27,8 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable.ArrayBuffer -import it.unimi.dsi.fastutil.ints.IntOpenHashSet - import org.apache.spark.Logging +import org.apache.spark.util.collection.OpenHashSet /** * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in @@ -207,7 +206,7 @@ private[spark] object SizeEstimator extends Logging { // Estimate the size of a large array by sampling elements without replacement. var size = 0.0 val rand = new Random(42) - val drawn = new IntOpenHashSet(ARRAY_SAMPLE_SIZE) + val drawn = new OpenHashSet[Int](ARRAY_SAMPLE_SIZE) for (i <- 0 until ARRAY_SAMPLE_SIZE) { var index = 0 do { 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 732748a7ff82b..d80eed455c427 100644 --- a/core/src/main/scala/org/apache/spark/util/StatCounter.scala +++ b/core/src/main/scala/org/apache/spark/util/StatCounter.scala @@ -62,10 +62,10 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { if (n == 0) { mu = other.mu m2 = other.m2 - n = other.n + n = other.n maxValue = other.maxValue minValue = other.minValue - } else if (other.n != 0) { + } else if (other.n != 0) { val delta = other.mu - mu if (other.n * 10 < n) { mu = mu + (delta * other.n) / (n + other.n) diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala index ddbd084ed7f01..8de75ba9a9c92 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala @@ -17,48 +17,54 @@ package org.apache.spark.util +import java.util.Set +import java.util.Map.Entry import java.util.concurrent.ConcurrentHashMap -import scala.collection.JavaConversions -import scala.collection.immutable -import scala.collection.mutable.Map +import scala.collection.{JavaConversions, mutable} import org.apache.spark.Logging +private[spark] case class TimeStampedValue[V](value: V, timestamp: Long) + /** * This is a custom implementation of scala.collection.mutable.Map which stores the insertion * timestamp along with each key-value pair. If specified, the timestamp of each pair can be * updated every time it is accessed. Key-value pairs whose timestamp are older than a particular * threshold time can then be removed using the clearOldValues method. This is intended to * be a drop-in replacement of scala.collection.mutable.HashMap. - * @param updateTimeStampOnGet When enabled, the timestamp of a pair will be - * updated when it is accessed + * + * @param updateTimeStampOnGet Whether timestamp of a pair will be updated when it is accessed */ -class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = false) - extends Map[A, B]() with Logging { - val internalMap = new ConcurrentHashMap[A, (B, Long)]() +private[spark] class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = false) + extends mutable.Map[A, B]() with Logging { + + private val internalMap = new ConcurrentHashMap[A, TimeStampedValue[B]]() def get(key: A): Option[B] = { val value = internalMap.get(key) if (value != null && updateTimeStampOnGet) { - internalMap.replace(key, value, (value._1, currentTime)) + internalMap.replace(key, value, TimeStampedValue(value.value, currentTime)) } - Option(value).map(_._1) + Option(value).map(_.value) } def iterator: Iterator[(A, B)] = { - val jIterator = internalMap.entrySet().iterator() - JavaConversions.asScalaIterator(jIterator).map(kv => (kv.getKey, kv.getValue._1)) + val jIterator = getEntrySet.iterator + JavaConversions.asScalaIterator(jIterator).map(kv => (kv.getKey, kv.getValue.value)) } - override def + [B1 >: B](kv: (A, B1)): Map[A, B1] = { + def getEntrySet: Set[Entry[A, TimeStampedValue[B]]] = internalMap.entrySet + + override def + [B1 >: B](kv: (A, B1)): mutable.Map[A, B1] = { val newMap = new TimeStampedHashMap[A, B1] - newMap.internalMap.putAll(this.internalMap) - newMap.internalMap.put(kv._1, (kv._2, currentTime)) + val oldInternalMap = this.internalMap.asInstanceOf[ConcurrentHashMap[A, TimeStampedValue[B1]]] + newMap.internalMap.putAll(oldInternalMap) + kv match { case (a, b) => newMap.internalMap.put(a, TimeStampedValue(b, currentTime)) } newMap } - override def - (key: A): Map[A, B] = { + override def - (key: A): mutable.Map[A, B] = { val newMap = new TimeStampedHashMap[A, B] newMap.internalMap.putAll(this.internalMap) newMap.internalMap.remove(key) @@ -66,17 +72,10 @@ class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = false) } override def += (kv: (A, B)): this.type = { - internalMap.put(kv._1, (kv._2, currentTime)) + kv match { case (a, b) => internalMap.put(a, TimeStampedValue(b, currentTime)) } this } - // Should we return previous value directly or as Option ? - def putIfAbsent(key: A, value: B): Option[B] = { - val prev = internalMap.putIfAbsent(key, (value, currentTime)) - if (prev != null) Some(prev._1) else None - } - - override def -= (key: A): this.type = { internalMap.remove(key) this @@ -87,53 +86,65 @@ class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = false) } override def apply(key: A): B = { - val value = internalMap.get(key) - if (value == null) throw new NoSuchElementException() - value._1 + get(key).getOrElse { throw new NoSuchElementException() } } - override def filter(p: ((A, B)) => Boolean): Map[A, B] = { - JavaConversions.mapAsScalaConcurrentMap(internalMap).map(kv => (kv._1, kv._2._1)).filter(p) + override def filter(p: ((A, B)) => Boolean): mutable.Map[A, B] = { + JavaConversions.mapAsScalaConcurrentMap(internalMap) + .map { case (k, TimeStampedValue(v, t)) => (k, v) } + .filter(p) } - override def empty: Map[A, B] = new TimeStampedHashMap[A, B]() + override def empty: mutable.Map[A, B] = new TimeStampedHashMap[A, B]() override def size: Int = internalMap.size override def foreach[U](f: ((A, B)) => U) { - val iterator = internalMap.entrySet().iterator() - while(iterator.hasNext) { - val entry = iterator.next() - val kv = (entry.getKey, entry.getValue._1) + val it = getEntrySet.iterator + while(it.hasNext) { + val entry = it.next() + val kv = (entry.getKey, entry.getValue.value) f(kv) } } - def toMap: immutable.Map[A, B] = iterator.toMap + def putIfAbsent(key: A, value: B): Option[B] = { + val prev = internalMap.putIfAbsent(key, TimeStampedValue(value, currentTime)) + Option(prev).map(_.value) + } + + def putAll(map: Map[A, B]) { + map.foreach { case (k, v) => update(k, v) } + } + + def toMap: Map[A, B] = iterator.toMap - /** - * Removes old key-value pairs that have timestamp earlier than `threshTime`, - * calling the supplied function on each such entry before removing. - */ def clearOldValues(threshTime: Long, f: (A, B) => Unit) { - val iterator = internalMap.entrySet().iterator() - while (iterator.hasNext) { - val entry = iterator.next() - if (entry.getValue._2 < threshTime) { - f(entry.getKey, entry.getValue._1) + val it = getEntrySet.iterator + while (it.hasNext) { + val entry = it.next() + if (entry.getValue.timestamp < threshTime) { + f(entry.getKey, entry.getValue.value) logDebug("Removing key " + entry.getKey) - iterator.remove() + it.remove() } } } - /** - * Removes old key-value pairs that have timestamp earlier than `threshTime` - */ + /** Removes old key-value pairs that have timestamp earlier than `threshTime`. */ def clearOldValues(threshTime: Long) { clearOldValues(threshTime, (_, _) => ()) } - private def currentTime: Long = System.currentTimeMillis() + private def currentTime: Long = System.currentTimeMillis + // For testing + + def getTimeStampedValue(key: A): Option[TimeStampedValue[B]] = { + Option(internalMap.get(key)) + } + + def getTimestamp(key: A): Option[Long] = { + getTimeStampedValue(key).map(_.timestamp) + } } diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala index 19bece86b36b4..7cd8f28b12dd6 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala @@ -22,7 +22,7 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConversions import scala.collection.mutable.Set -class TimeStampedHashSet[A] extends Set[A] { +private[spark] class TimeStampedHashSet[A] extends Set[A] { val internalMap = new ConcurrentHashMap[A, Long]() def contains(key: A): Boolean = { diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala new file mode 100644 index 0000000000000..f5be5856c2109 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.lang.ref.WeakReference +import java.util.concurrent.atomic.AtomicInteger + +import scala.collection.mutable +import scala.language.implicitConversions + +import org.apache.spark.Logging + +/** + * A wrapper of TimeStampedHashMap that ensures the values are weakly referenced and timestamped. + * + * If the value is garbage collected and the weak reference is null, get() will return a + * non-existent value. These entries are removed from the map periodically (every N inserts), as + * their values are no longer strongly reachable. Further, key-value pairs whose timestamps are + * older than a particular threshold can be removed using the clearOldValues method. + * + * TimeStampedWeakValueHashMap exposes a scala.collection.mutable.Map interface, which allows it + * to be a drop-in replacement for Scala HashMaps. Internally, it uses a Java ConcurrentHashMap, + * so all operations on this HashMap are thread-safe. + * + * @param updateTimeStampOnGet Whether timestamp of a pair will be updated when it is accessed. + */ +private[spark] class TimeStampedWeakValueHashMap[A, B](updateTimeStampOnGet: Boolean = false) + extends mutable.Map[A, B]() with Logging { + + import TimeStampedWeakValueHashMap._ + + private val internalMap = new TimeStampedHashMap[A, WeakReference[B]](updateTimeStampOnGet) + private val insertCount = new AtomicInteger(0) + + /** Return a map consisting only of entries whose values are still strongly reachable. */ + private def nonNullReferenceMap = internalMap.filter { case (_, ref) => ref.get != null } + + def get(key: A): Option[B] = internalMap.get(key) + + def iterator: Iterator[(A, B)] = nonNullReferenceMap.iterator + + override def + [B1 >: B](kv: (A, B1)): mutable.Map[A, B1] = { + val newMap = new TimeStampedWeakValueHashMap[A, B1] + val oldMap = nonNullReferenceMap.asInstanceOf[mutable.Map[A, WeakReference[B1]]] + newMap.internalMap.putAll(oldMap.toMap) + newMap.internalMap += kv + newMap + } + + override def - (key: A): mutable.Map[A, B] = { + val newMap = new TimeStampedWeakValueHashMap[A, B] + newMap.internalMap.putAll(nonNullReferenceMap.toMap) + newMap.internalMap -= key + newMap + } + + override def += (kv: (A, B)): this.type = { + internalMap += kv + if (insertCount.incrementAndGet() % CLEAR_NULL_VALUES_INTERVAL == 0) { + clearNullValues() + } + this + } + + override def -= (key: A): this.type = { + internalMap -= key + this + } + + override def update(key: A, value: B) = this += ((key, value)) + + override def apply(key: A): B = internalMap.apply(key) + + override def filter(p: ((A, B)) => Boolean): mutable.Map[A, B] = nonNullReferenceMap.filter(p) + + override def empty: mutable.Map[A, B] = new TimeStampedWeakValueHashMap[A, B]() + + override def size: Int = internalMap.size + + override def foreach[U](f: ((A, B)) => U) = 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) + + /** Remove entries with values that are no longer strongly reachable. */ + def clearNullValues() { + val it = internalMap.getEntrySet.iterator + while (it.hasNext) { + val entry = it.next() + if (entry.getValue.value.get == null) { + logDebug("Removing key " + entry.getKey + " because it is no longer strongly reachable.") + it.remove() + } + } + } + + // For testing + + def getTimestamp(key: A): Option[Long] = { + internalMap.getTimeStampedValue(key).map(_.timestamp) + } + + def getReference(key: A): Option[WeakReference[B]] = { + internalMap.getTimeStampedValue(key).map(_.value) + } +} + +/** + * Helper methods for converting to and from WeakReferences. + */ +private object TimeStampedWeakValueHashMap { + + // Number of inserts after which entries with null references are removed + val CLEAR_NULL_VALUES_INTERVAL = 100 + + /* Implicit conversion methods to WeakReferences. */ + + implicit def toWeakReference[V](v: V): WeakReference[V] = new WeakReference[V](v) + + implicit def toWeakReferenceTuple[K, V](kv: (K, V)): (K, WeakReference[V]) = { + kv match { case (k, v) => (k, toWeakReference(v)) } + } + + implicit def toWeakReferenceFunction[K, V, R](p: ((K, V)) => R): ((K, WeakReference[V])) => R = { + (kv: (K, WeakReference[V])) => p(kv) + } + + /* Implicit conversion methods from WeakReferences. */ + + implicit def fromWeakReference[V](ref: WeakReference[V]): V = ref.get + + implicit def fromWeakReferenceOption[V](v: Option[WeakReference[V]]): Option[V] = { + v match { + case Some(ref) => Option(fromWeakReference(ref)) + case None => None + } + } + + implicit def fromWeakReferenceTuple[K, V](kv: (K, WeakReference[V])): (K, V) = { + kv match { case (k, v) => (k, fromWeakReference(v)) } + } + + implicit def fromWeakReferenceIterator[K, V]( + it: Iterator[(K, WeakReference[V])]): Iterator[(K, V)] = { + it.map(fromWeakReferenceTuple) + } + + implicit def fromWeakReferenceMap[K, V]( + map: mutable.Map[K, WeakReference[V]]) : mutable.Map[K, V] = { + mutable.Map(map.mapValues(fromWeakReference).toSeq: _*) + } +} 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 d3c39dee330b2..a2454e120a8ab 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -26,27 +26,36 @@ import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor} import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.SortedSet import scala.io.Source import scala.reflect.ClassTag +import scala.util.Try +import scala.util.control.{ControlThrowable, NonFatal} import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder +import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} import org.json4s._ import tachyon.client.{TachyonFile,TachyonFS} import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.executor.ExecutorUncaughtExceptionHandler import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} +/** CallSite represents a place in user code. It can have a short and a long form. */ +private[spark] case class CallSite(val short: String, val long: String) /** * Various utility methods used by Spark. */ private[spark] object Utils extends Logging { + val random = new Random() - val osName = System.getProperty("os.name") + def sparkBin(sparkHome: String, which: String): File = { + val suffix = if (isWindows) ".cmd" else "" + new File(sparkHome + File.separator + "bin", which + suffix) + } /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { @@ -117,6 +126,26 @@ private[spark] object Utils extends Logging { } } + /** + * Get the ClassLoader which loaded Spark. + */ + def getSparkClassLoader = getClass.getClassLoader + + /** + * Get the Context ClassLoader on this thread or, if not present, the ClassLoader that + * loaded Spark. + * + * 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 = + Option(Thread.currentThread().getContextClassLoader).getOrElse(getSparkClassLoader) + + /** Determines whether the provided class is loadable in the current thread. */ + def classIsLoadable(clazz: String): Boolean = { + Try { Class.forName(clazz, false, getContextOrSparkClassLoader) }.isSuccess + } + /** * Primitive often used when writing {@link java.nio.ByteBuffer} to {@link java.io.DataOutput}. */ @@ -195,9 +224,9 @@ private[spark] object Utils extends Logging { def hasRootAsShutdownDeleteDir(file: File): Boolean = { val absolutePath = file.getAbsolutePath() val retval = shutdownDeletePaths.synchronized { - shutdownDeletePaths.find { path => + shutdownDeletePaths.exists { path => !absolutePath.equals(path) && absolutePath.startsWith(path) - }.isDefined + } } if (retval) { logInfo("path = " + file + ", already present as root for deletion.") @@ -210,10 +239,10 @@ private[spark] object Utils extends Logging { // paths - resulting in Exception and incomplete cleanup. def hasRootAsShutdownDeleteDir(file: TachyonFile): Boolean = { val absolutePath = file.getPath() - val retval = shutdownDeletePaths.synchronized { - shutdownDeletePaths.find { path => + val retval = shutdownDeleteTachyonPaths.synchronized { + shutdownDeleteTachyonPaths.exists { path => !absolutePath.equals(path) && absolutePath.startsWith(path) - }.isDefined + } } if (retval) { logInfo("path = " + file + ", already present as root for deletion.") @@ -499,10 +528,10 @@ private[spark] object Utils extends Logging { private val hostPortParseResults = new ConcurrentHashMap[String, (String, Int)]() def parseHostPort(hostPort: String): (String, Int) = { - { - // Check cache first. - val cached = hostPortParseResults.get(hostPort) - if (cached != null) return cached + // Check cache first. + val cached = hostPortParseResults.get(hostPort) + if (cached != null) { + return cached } val indx: Int = hostPort.lastIndexOf(':') @@ -533,8 +562,7 @@ private[spark] object Utils extends Logging { } /** - * Return the string to tell how long has passed in seconds. The passing parameter should be in - * millisecond. + * Return the string to tell how long has passed in milliseconds. */ def getUsedTimeMs(startTimeMs: Long): String = { " " + (System.currentTimeMillis - startTimeMs) + " ms" @@ -562,15 +590,17 @@ private[spark] object Utils extends Logging { * Don't follow directories if they are symlinks. */ def deleteRecursively(file: File) { - if ((file.isDirectory) && !isSymlink(file)) { - for (child <- listFilesSafely(file)) { - deleteRecursively(child) + if (file != null) { + if ((file.isDirectory) && !isSymlink(file)) { + for (child <- listFilesSafely(file)) { + deleteRecursively(child) + } } - } - if (!file.delete()) { - // Delete can also fail if the file simply did not exist - if (file.exists()) { - throw new IOException("Failed to delete: " + file.getAbsolutePath) + if (!file.delete()) { + // Delete can also fail if the file simply did not exist + if (file.exists()) { + throw new IOException("Failed to delete: " + file.getAbsolutePath) + } } } } @@ -589,7 +619,7 @@ private[spark] object Utils extends Logging { */ def isSymlink(file: File): Boolean = { if (file == null) throw new NullPointerException("File must not be null") - if (osName.startsWith("Windows")) return false + if (isWindows) return false val fileInCanonicalDir = if (file.getParent() == null) { file } else { @@ -597,9 +627,24 @@ private[spark] object Utils extends Logging { } if (fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile())) { - return false; + return false + } else { + return true + } + } + + /** + * Finds all the files in a directory whose last modified time is older than cutoff seconds. + * @param dir must be the path to a directory, or IllegalArgumentException is thrown + * @param cutoff measured in seconds. Files older than this are returned. + */ + def findOldFiles(dir: File, cutoff: Long): Seq[File] = { + val currentTimeMillis = System.currentTimeMillis + if (dir.isDirectory) { + val files = listFilesSafely(dir) + files.filter { file => file.lastModified < (currentTimeMillis - cutoff * 1000) } } else { - return true; + throw new IllegalArgumentException(dir + " is not a directory!") } } @@ -739,27 +784,30 @@ private[spark] object Utils extends Logging { output.toString } + /** + * Execute a block of code that evaluates to Unit, forwarding any uncaught exceptions to the + * default UncaughtExceptionHandler + */ + def tryOrExit(block: => Unit) { + try { + block + } catch { + case t: Throwable => ExecutorUncaughtExceptionHandler.uncaughtException(t) + } + } + /** * 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 val SPARK_CLASS_REGEX = """^org\.apache\.spark(\.api\.java)?(\.util)?(\.rdd)?\.[A-Z]""".r - private[spark] class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String, - val firstUserLine: Int, val firstUserClass: String) { - - /** Returns a printable version of the call site info suitable for logs. */ - override def toString = { - "%s at %s:%s".format(lastSparkMethod, firstUserFile, firstUserLine) - } - } - /** * When called inside a class in the spark package, returns the name of the user code class * (outside the spark package) that called into Spark, as well as which Spark method they called. * This is used, for example, to tell users where in their code each RDD got created. */ - def getCallSiteInfo: CallSiteInfo = { + def getCallSite: CallSite = { val trace = Thread.currentThread.getStackTrace() .filterNot(_.getMethodName.contains("getStackTrace")) @@ -770,11 +818,11 @@ private[spark] object Utils extends Logging { var lastSparkMethod = "" var firstUserFile = "" var firstUserLine = 0 - var finished = false - var firstUserClass = "" + var insideSpark = true + var callStack = new ArrayBuffer[String]() :+ "" for (el <- trace) { - if (!finished) { + if (insideSpark) { if (SPARK_CLASS_REGEX.findFirstIn(el.getClassName).isDefined) { lastSparkMethod = if (el.getMethodName == "") { // Spark method is a constructor; get its class name @@ -782,16 +830,21 @@ private[spark] object Utils extends Logging { } else { el.getMethodName } - } - else { + callStack(0) = el.toString // Put last Spark method on top of the stack trace. + } else { firstUserLine = el.getLineNumber firstUserFile = el.getFileName - firstUserClass = el.getClassName - finished = true + callStack += el.toString + insideSpark = false } + } else { + callStack += el.toString } } - new CallSiteInfo(lastSparkMethod, firstUserFile, firstUserLine, firstUserClass) + val callStackDepth = System.getProperty("spark.callstack.depth", "20").toInt + CallSite( + short = "%s at %s:%s".format(lastSparkMethod, firstUserFile, firstUserLine), + long = callStack.take(callStackDepth).mkString("\n")) } /** Return a string containing part of a file from byte 'start' to 'end'. */ @@ -809,10 +862,63 @@ private[spark] object Utils extends Logging { Source.fromBytes(buff).mkString } + /** + * Return a string containing data across a set of files. The `startIndex` + * and `endIndex` is based on the cumulative size of all the files take in + * the given order. See figure below for more details. + */ + def offsetBytes(files: Seq[File], start: Long, end: Long): String = { + val fileLengths = files.map { _.length } + val startIndex = math.max(start, 0) + val endIndex = math.min(end, fileLengths.sum) + val fileToLength = files.zip(fileLengths).toMap + logDebug("Log files: \n" + fileToLength.mkString("\n")) + + val stringBuffer = new StringBuffer((endIndex - startIndex).toInt) + var sum = 0L + for (file <- files) { + val startIndexOfFile = sum + val endIndexOfFile = sum + fileToLength(file) + logDebug(s"Processing file $file, " + + s"with start index = $startIndexOfFile, end index = $endIndex") + + /* + ____________ + range 1: | | + | case A | + + files: |==== file 1 ====|====== file 2 ======|===== file 3 =====| + + | case B . case C . case D | + range 2: |___________.____________________.______________| + */ + + if (startIndex <= startIndexOfFile && endIndex >= endIndexOfFile) { + // Case C: read the whole file + stringBuffer.append(offsetBytes(file.getAbsolutePath, 0, fileToLength(file))) + } else if (startIndex > startIndexOfFile && startIndex < endIndexOfFile) { + // Case A and B: read from [start of required range] to [end of file / end of range] + val effectiveStartIndex = startIndex - startIndexOfFile + val effectiveEndIndex = math.min(endIndex - startIndexOfFile, fileToLength(file)) + stringBuffer.append(Utils.offsetBytes( + file.getAbsolutePath, effectiveStartIndex, effectiveEndIndex)) + } else if (endIndex > startIndexOfFile && endIndex < endIndexOfFile) { + // Case D: read from [start of file] to [end of require range] + val effectiveStartIndex = math.max(startIndex - startIndexOfFile, 0) + val effectiveEndIndex = endIndex - startIndexOfFile + stringBuffer.append(Utils.offsetBytes( + file.getAbsolutePath, effectiveStartIndex, effectiveEndIndex)) + } + sum += fileToLength(file) + logDebug(s"After processing file $file, string built is ${stringBuffer.toString}}") + } + stringBuffer.toString + } + /** * Clone an object using a Spark serializer. */ - def clone[T](value: T, serializer: SerializerInstance): T = { + def clone[T: ClassTag](value: T, serializer: SerializerInstance): T = { serializer.deserialize[T](serializer.serialize(value)) } @@ -978,10 +1084,18 @@ private[spark] object Utils extends Logging { if (dst.isAbsolute()) { throw new IOException("Destination must be relative") } - val linkCmd = if (osName.startsWith("Windows")) "copy" else "ln -sf" + var cmdSuffix = "" + val linkCmd = if (isWindows) { + // refer to http://technet.microsoft.com/en-us/library/cc771254.aspx + cmdSuffix = " /s /e /k /h /y /i" + "cmd /c xcopy " + } else { + cmdSuffix = "" + "ln -sf " + } import scala.sys.process._ - (linkCmd + " " + src.getAbsolutePath() + " " + dst.getPath()) lines_! ProcessLogger(line => - (logInfo(line))) + (linkCmd + src.getAbsolutePath() + " " + dst.getPath() + cmdSuffix) lines_! + ProcessLogger(line => (logInfo(line))) } @@ -1007,4 +1121,169 @@ private[spark] object Utils extends Logging { def getHadoopFileSystem(path: URI): FileSystem = { FileSystem.get(path, SparkHadoopUtil.get.newConfiguration()) } + + /** + * Return a Hadoop FileSystem with the scheme encoded in the given path. + */ + def getHadoopFileSystem(path: String): FileSystem = { + getHadoopFileSystem(new URI(path)) + } + + /** + * Return the absolute path of a file in the given directory. + */ + def getFilePath(dir: File, fileName: String): Path = { + assert(dir.isDirectory) + val path = new File(dir, fileName).getAbsolutePath + new Path(path) + } + + /** + * Whether the underlying operating system is Windows. + */ + val isWindows = SystemUtils.IS_OS_WINDOWS + + /** + * Pattern for matching a Windows drive, which contains only a single alphabet character. + */ + val windowsDrive = "([a-zA-Z])".r + + /** + * Format a Windows path such that it can be safely passed to a URI. + */ + def formatWindowsPath(path: String): String = path.replace("\\", "/") + + /** + * Indicates whether Spark is currently running unit tests. + */ + def isTesting = { + sys.env.contains("SPARK_TESTING") || sys.props.contains("spark.testing") + } + + /** + * Strip the directory from a path name + */ + def stripDirectory(path: String): String = { + new File(path).getName + } + + /** + * Wait for a process to terminate for at most the specified duration. + * Return whether the process actually terminated after the given timeout. + */ + def waitForProcess(process: Process, timeoutMs: Long): Boolean = { + var terminated = false + val startTime = System.currentTimeMillis + while (!terminated) { + try { + process.exitValue + terminated = true + } catch { + case e: IllegalThreadStateException => + // Process not terminated yet + if (System.currentTimeMillis - startTime > timeoutMs) { + return false + } + Thread.sleep(100) + } + } + true + } + + /** + * Return the stderr of a process after waiting for the process to terminate. + * If the process does not terminate within the specified timeout, return None. + */ + def getStderr(process: Process, timeoutMs: Long): Option[String] = { + val terminated = Utils.waitForProcess(process, timeoutMs) + if (terminated) { + Some(Source.fromInputStream(process.getErrorStream).getLines().mkString("\n")) + } else { + None + } + } + + /** + * Execute the given block, logging and re-throwing any uncaught exception. + * This is particularly useful for wrapping code that runs in a thread, to ensure + * that exceptions are printed, and to avoid having to catch Throwable. + */ + def logUncaughtExceptions[T](f: => T): T = { + try { + f + } catch { + case ct: ControlThrowable => + throw ct + case t: Throwable => + logError(s"Uncaught exception in thread ${Thread.currentThread().getName}", t) + throw t + } + } + + /** Returns true if the given exception was fatal. See docs for scala.util.control.NonFatal. */ + def isFatalError(e: Throwable): Boolean = { + e match { + case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable => + false + case _ => + true + } + } + + /** + * Return a well-formed URI for the file described by a user input string. + * + * If the supplied path does not contain a scheme, or is a relative path, it will be + * converted into an absolute path with a file:// scheme. + */ + def resolveURI(path: String, testWindows: Boolean = false): URI = { + + // In Windows, the file separator is a backslash, but this is inconsistent with the URI format + val windows = isWindows || testWindows + val formattedPath = if (windows) formatWindowsPath(path) else path + + val uri = new URI(formattedPath) + if (uri.getPath == null) { + throw new IllegalArgumentException(s"Given path is malformed: $uri") + } + uri.getScheme match { + case windowsDrive(d) if windows => + new URI("file:/" + uri.toString.stripPrefix("/")) + case null => + // Preserve fragments for HDFS file name substitution (denoted by "#") + // For instance, in "abc.py#xyz.py", "xyz.py" is the name observed by the application + val fragment = uri.getFragment + val part = new File(uri.getPath).toURI + new URI(part.getScheme, part.getPath, fragment) + case _ => + uri + } + } + + /** Resolve a comma-separated list of paths. */ + def resolveURIs(paths: String, testWindows: Boolean = false): String = { + if (paths == null || paths.trim.isEmpty) { + "" + } else { + paths.split(",").map { p => Utils.resolveURI(p, testWindows) }.mkString(",") + } + } + + /** Return all non-local paths from a comma-separated list of paths. */ + def nonLocalPaths(paths: String, testWindows: Boolean = false): Array[String] = { + val windows = isWindows || testWindows + if (paths == null || paths.trim.isEmpty) { + Array.empty + } else { + paths.split(",").filter { p => + val formattedPath = if (windows) formatWindowsPath(p) else p + new URI(formattedPath).getScheme match { + case windowsDrive(d) if windows => false + case "local" | "file" | null => false + case _ => true + } + } + } + } + } diff --git a/core/src/main/scala/org/apache/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala index dc4b8f253f259..c6cab82c3e546 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -17,10 +17,12 @@ package org.apache.spark.util +import scala.language.implicitConversions import scala.util.Random import org.apache.spark.util.random.XORShiftRandom +@deprecated("Use Vectors.dense from Spark's mllib.linalg package instead.", "1.0.0") class Vector(val elements: Array[Double]) extends Serializable { def length = elements.length @@ -135,7 +137,7 @@ object Vector { def ones(length: Int) = Vector(length, _ => 1) /** - * Creates this [[org.apache.spark.util.Vector]] of given length containing random numbers + * Creates this [[org.apache.spark.util.Vector]] of given length containing random numbers * between 0.0 and 1.0. Optional scala.util.Random number generator can be provided. */ def random(length: Int, random: Random = new XORShiftRandom()) = diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index b8c852b4ff5c7..1a6f1c2b55799 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -19,7 +19,12 @@ package org.apache.spark.util.collection import java.util.{Arrays, Comparator} +import com.google.common.hash.Hashing + +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A simple open hash table optimized for the append-only use case, where keys * are never removed, but the value for each key may be changed. * @@ -29,12 +34,14 @@ import java.util.{Arrays, Comparator} * * TODO: Cache the hash values of each key? java.util.HashMap does that. */ -private[spark] -class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, - V)] with Serializable { +@DeveloperApi +class AppendOnlyMap[K, V](initialCapacity: Int = 64) + extends Iterable[(K, V)] with Serializable { require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements") require(initialCapacity >= 1, "Invalid initial capacity") + private val LOAD_FACTOR = 0.7 + private var capacity = nextPowerOf2(initialCapacity) private var mask = capacity - 1 private var curSize = 0 @@ -52,8 +59,6 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, private var destroyed = false private val destructionMessage = "Map state is invalid from destructive sorting!" - private val LOAD_FACTOR = 0.7 - /** Get the value for a given key */ def apply(key: K): V = { assert(!destroyed, destructionMessage) @@ -196,11 +201,8 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, /** * Re-hash a value to deal better with hash functions that don't differ in the lower bits. - * We use the Murmur Hash 3 finalization step that's also used in fastutil. */ - private def rehash(h: Int): Int = { - it.unimi.dsi.fastutil.HashCommon.murmurHash3(h) - } + private def rehash(h: Int): Int = Hashing.murmur3_32().hashInt(h).asInt() /** Double the table's size and re-hash everything */ protected def growTable() { 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 caa06d5b445b4..288badd3160f8 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 @@ -17,20 +17,22 @@ package org.apache.spark.util.collection -import java.io._ +import java.io.{InputStream, BufferedInputStream, FileInputStream, File, Serializable, EOFException} import java.util.Comparator +import scala.collection.BufferedIterator import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import com.google.common.io.ByteStreams -import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BlockId, BlockManager} /** + * :: DeveloperApi :: * An append-only map that spills sorted content to disk when there is insufficient space for it * to grow. * @@ -55,8 +57,8 @@ import org.apache.spark.storage.{BlockId, BlockManager} * `spark.shuffle.safetyFraction` specifies an additional margin of safety as a fraction of * this threshold, in case map size estimation is not sufficiently accurate. */ - -private[spark] class ExternalAppendOnlyMap[K, V, C]( +@DeveloperApi +class ExternalAppendOnlyMap[K, V, C]( createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, @@ -230,7 +232,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( // Input streams are derived both from the in-memory map and spilled maps on disk // The in-memory map is sorted in place, while the spilled maps are already in sorted order private val sortedMap = currentMap.destructiveSortedIterator(comparator) - private val inputStreams = Seq(sortedMap) ++ spilledMaps + private val inputStreams = (Seq(sortedMap) ++ spilledMaps).map(it => it.buffered) inputStreams.foreach { it => val kcPairs = getMorePairs(it) @@ -245,13 +247,13 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( * In the event of key hash collisions, this ensures no pairs are hidden from being merged. * Assume the given iterator is in sorted order. */ - private def getMorePairs(it: Iterator[(K, C)]): ArrayBuffer[(K, C)] = { + private def getMorePairs(it: BufferedIterator[(K, C)]): ArrayBuffer[(K, C)] = { val kcPairs = new ArrayBuffer[(K, C)] if (it.hasNext) { var kc = it.next() kcPairs += kc val minHash = kc._1.hashCode() - while (it.hasNext && kc._1.hashCode() == minHash) { + while (it.hasNext && it.head._1.hashCode() == minHash) { kc = it.next() kcPairs += kc } @@ -324,7 +326,8 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( * * StreamBuffers are ordered by the minimum key hash found across all of their own pairs. */ - private case class StreamBuffer(iterator: Iterator[(K, C)], pairs: ArrayBuffer[(K, C)]) + private class StreamBuffer( + val iterator: BufferedIterator[(K, C)], val pairs: ArrayBuffer[(K, C)]) extends Comparable[StreamBuffer] { def isEmpty = pairs.length == 0 @@ -336,8 +339,8 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( } override def compareTo(other: StreamBuffer): Int = { - // minus sign because mutable.PriorityQueue dequeues the max, not the min - -minKeyHash.compareTo(other.minKeyHash) + // descending order because mutable.PriorityQueue dequeues the max, not the min + if (other.minKeyHash < minKeyHash) -1 else if (other.minKeyHash == minKeyHash) 0 else 1 } } } @@ -348,7 +351,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( private class DiskMapIterator(file: File, blockId: BlockId, batchSizes: ArrayBuffer[Long]) extends Iterator[(K, C)] { private val fileStream = new FileInputStream(file) - private val bufferedStream = new FastBufferedInputStream(fileStream, fileBufferSize) + private val bufferedStream = new BufferedInputStream(fileStream, fileBufferSize) // An intermediate stream that reads from exactly one batch // This guards against pre-fetching and other arbitrary behavior of higher level streams @@ -421,7 +424,9 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( private[spark] object ExternalAppendOnlyMap { private class KCComparator[K, C] extends Comparator[(K, C)] { def compare(kc1: (K, C), kc2: (K, C)): Int = { - kc1._1.hashCode().compareTo(kc2._1.hashCode()) + val hash1 = kc1._1.hashCode() + val hash2 = kc2._1.hashCode() + if (hash1 < hash2) -1 else if (hash1 == hash2) 0 else 1 } } } 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 c26f23d50024a..b8de4ff9aa494 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 @@ -19,15 +19,19 @@ package org.apache.spark.util.collection import scala.reflect.ClassTag +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A fast hash map implementation for nullable keys. This hash map supports insertions and updates, * but not deletions. This map is about 5X faster than java.util.HashMap, while using much less * space overhead. * * Under the hood, it uses our OpenHashSet implementation. */ +@DeveloperApi private[spark] -class OpenHashMap[K >: Null : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( +class OpenHashMap[K : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( initialCapacity: Int) extends Iterable[(K, V)] with Serializable { 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 148c12e64d2ce..4e363b74f4bef 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 @@ -18,6 +18,7 @@ package org.apache.spark.util.collection import scala.reflect._ +import com.google.common.hash.Hashing /** * A simple, fast hash set optimized for non-null insertion-only use case, where keys are never @@ -114,10 +115,10 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag]( * The caller is responsible for calling rehashIfNeeded. * * Use (retval & POSITION_MASK) to get the actual position, and - * (retval & EXISTENCE_MASK) != 0 for prior existence. + * (retval & NONEXISTENCE_MASK) == 0 for prior existence. * * @return The position where the key is placed, plus the highest order bit is set if the key - * exists previously. + * does not exists previously. */ def addWithoutResize(k: T): Int = { var pos = hashcode(hasher.hash(k)) & _mask @@ -256,9 +257,8 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag]( /** * Re-hash a value to deal better with hash functions that don't differ in the lower bits. - * We use the Murmur Hash 3 finalization step that's also used in fastutil. */ - private def hashcode(h: Int): Int = it.unimi.dsi.fastutil.HashCommon.murmurHash3(h) + private def hashcode(h: Int): Int = Hashing.murmur3_32().hashInt(h).asInt() private def nextPowerOf2(n: Int): Int = { val highBit = Integer.highestOneBit(n) 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 new file mode 100644 index 0000000000000..8e9c3036d09c2 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.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.util.logging + +import java.io.{File, FileOutputStream, InputStream} + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.util.{IntParam, Utils} + +/** + * Continuously appends the data from an input stream into the given file. + */ +private[spark] class FileAppender(inputStream: InputStream, file: File, bufferSize: Int = 8192) + extends Logging { + @volatile private var outputStream: FileOutputStream = null + @volatile private var markedForStop = false // has the appender been asked to stopped + @volatile private var stopped = false // has the appender stopped + + // Thread that reads the input stream and writes to file + private val writingThread = new Thread("File appending thread for " + file) { + setDaemon(true) + override def run() { + Utils.logUncaughtExceptions { + appendStreamToFile() + } + } + } + writingThread.start() + + /** + * Wait for the appender to stop appending, either because input stream is closed + * or because of any error in appending + */ + def awaitTermination() { + synchronized { + if (!stopped) { + wait() + } + } + } + + /** Stop the appender */ + def stop() { + markedForStop = true + } + + /** Continuously read chunks from the input stream and append to the file */ + protected def appendStreamToFile() { + try { + logDebug("Started appending thread") + openFile() + val buf = new Array[Byte](bufferSize) + var n = 0 + while (!markedForStop && n != -1) { + n = inputStream.read(buf) + if (n != -1) { + appendToFile(buf, n) + } + } + } catch { + case e: Exception => + logError(s"Error writing stream to file $file", e) + } finally { + closeFile() + synchronized { + stopped = true + notifyAll() + } + } + } + + /** Append bytes to the file output stream */ + protected def appendToFile(bytes: Array[Byte], len: Int) { + if (outputStream == null) { + openFile() + } + outputStream.write(bytes, 0, len) + } + + /** Open the file output stream */ + protected def openFile() { + outputStream = new FileOutputStream(file, false) + logDebug(s"Opened file $file") + } + + /** Close the file output stream */ + protected def closeFile() { + outputStream.flush() + outputStream.close() + logDebug(s"Closed file $file") + } +} + +/** + * Companion object to [[org.apache.spark.util.logging.FileAppender]] which has helper + * functions to choose the correct type of FileAppender based on SparkConf configuration. + */ +private[spark] object FileAppender extends Logging { + + /** Create the right appender based on Spark configuration */ + def apply(inputStream: InputStream, file: File, conf: SparkConf): FileAppender = { + + import RollingFileAppender._ + + val rollingStrategy = conf.get(STRATEGY_PROPERTY, STRATEGY_DEFAULT) + val rollingSizeBytes = conf.get(SIZE_PROPERTY, STRATEGY_DEFAULT) + val rollingInterval = conf.get(INTERVAL_PROPERTY, INTERVAL_DEFAULT) + + def createTimeBasedAppender() = { + val validatedParams: Option[(Long, String)] = rollingInterval match { + case "daily" => + logInfo(s"Rolling executor logs enabled for $file with daily rolling") + Some(24 * 60 * 60 * 1000L, "--YYYY-MM-dd") + case "hourly" => + logInfo(s"Rolling executor logs enabled for $file with hourly rolling") + Some(60 * 60 * 1000L, "--YYYY-MM-dd--HH") + case "minutely" => + logInfo(s"Rolling executor logs enabled for $file with rolling every minute") + Some(60 * 1000L, "--YYYY-MM-dd--HH-mm") + case IntParam(seconds) => + logInfo(s"Rolling executor logs enabled for $file with rolling $seconds seconds") + Some(seconds * 1000L, "--YYYY-MM-dd--HH-mm-ss") + case _ => + logWarning(s"Illegal interval for rolling executor logs [$rollingInterval], " + + s"rolling logs not enabled") + None + } + validatedParams.map { + case (interval, pattern) => + new RollingFileAppender( + inputStream, file, new TimeBasedRollingPolicy(interval, pattern), conf) + }.getOrElse { + new FileAppender(inputStream, file) + } + } + + def createSizeBasedAppender() = { + rollingSizeBytes match { + case IntParam(bytes) => + logInfo(s"Rolling executor logs enabled for $file with rolling every $bytes bytes") + new RollingFileAppender(inputStream, file, new SizeBasedRollingPolicy(bytes), conf) + case _ => + logWarning( + s"Illegal size [$rollingSizeBytes] for rolling executor logs, rolling logs not enabled") + new FileAppender(inputStream, file) + } + } + + rollingStrategy match { + case "" => + new FileAppender(inputStream, file) + case "time" => + createTimeBasedAppender() + case "size" => + createSizeBasedAppender() + case _ => + logWarning( + s"Illegal strategy [$rollingStrategy] for rolling executor logs, " + + s"rolling logs not enabled") + new FileAppender(inputStream, file) + } + } +} + + 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 new file mode 100644 index 0000000000000..1bbbd20cf076f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.logging + +import java.io.{File, FileFilter, InputStream} + +import org.apache.commons.io.FileUtils +import org.apache.spark.SparkConf +import RollingFileAppender._ + +/** + * Continuously appends data from input stream into the given file, and rolls + * over the file after the given interval. The rolled over files are named + * based on the given pattern. + * + * @param inputStream Input stream to read data from + * @param activeFile File to write data to + * @param rollingPolicy Policy based on which files will be rolled over. + * @param conf SparkConf that is used to pass on extra configurations + * @param bufferSize Optional buffer size. Used mainly for testing. + */ +private[spark] class RollingFileAppender( + inputStream: InputStream, + activeFile: File, + val rollingPolicy: RollingPolicy, + conf: SparkConf, + bufferSize: Int = DEFAULT_BUFFER_SIZE + ) extends FileAppender(inputStream, activeFile, bufferSize) { + + private val maxRetainedFiles = conf.getInt(RETAINED_FILES_PROPERTY, -1) + + /** Stop the appender */ + override def stop() { + super.stop() + } + + /** Append bytes to file after rolling over is necessary */ + override protected def appendToFile(bytes: Array[Byte], len: Int) { + if (rollingPolicy.shouldRollover(len)) { + rollover() + rollingPolicy.rolledOver() + } + super.appendToFile(bytes, len) + rollingPolicy.bytesWritten(len) + } + + /** Rollover the file, by closing the output stream and moving it over */ + private def rollover() { + try { + closeFile() + moveFile() + openFile() + if (maxRetainedFiles > 0) { + deleteOldFiles() + } + } catch { + case e: Exception => + logError(s"Error rolling over $activeFile", e) + } + } + + /** Move the active log file to a new rollover file */ + private def moveFile() { + val rolloverSuffix = rollingPolicy.generateRolledOverFileSuffix() + val rolloverFile = new File( + activeFile.getParentFile, activeFile.getName + rolloverSuffix).getAbsoluteFile + try { + logDebug(s"Attempting to rollover file $activeFile to file $rolloverFile") + if (activeFile.exists) { + if (!rolloverFile.exists) { + FileUtils.moveFile(activeFile, rolloverFile) + logInfo(s"Rolled over $activeFile to $rolloverFile") + } else { + // In case the rollover file name clashes, make a unique file name. + // The resultant file names are long and ugly, so this is used only + // if there is a name collision. This can be avoided by the using + // the right pattern such that name collisions do not occur. + var i = 0 + var altRolloverFile: File = null + do { + altRolloverFile = new File(activeFile.getParent, + s"${activeFile.getName}$rolloverSuffix--$i").getAbsoluteFile + i += 1 + } while (i < 10000 && altRolloverFile.exists) + + logWarning(s"Rollover file $rolloverFile already exists, " + + s"rolled over $activeFile to file $altRolloverFile") + FileUtils.moveFile(activeFile, altRolloverFile) + } + } else { + logWarning(s"File $activeFile does not exist") + } + } + } + + /** Retain only last few files */ + private[util] def deleteOldFiles() { + try { + val rolledoverFiles = activeFile.getParentFile.listFiles(new FileFilter { + def accept(f: File): Boolean = { + f.getName.startsWith(activeFile.getName) && f != activeFile + } + }).sorted + val filesToBeDeleted = rolledoverFiles.take( + math.max(0, rolledoverFiles.size - maxRetainedFiles)) + filesToBeDeleted.foreach { file => + logInfo(s"Deleting file executor log file ${file.getAbsolutePath}") + file.delete() + } + } catch { + case e: Exception => + logError("Error cleaning logs in directory " + activeFile.getParentFile.getAbsolutePath, e) + } + } +} + +/** + * Companion object to [[org.apache.spark.util.logging.RollingFileAppender]]. Defines + * names of configurations that configure rolling file appenders. + */ +private[spark] object RollingFileAppender { + val STRATEGY_PROPERTY = "spark.executor.logs.rolling.strategy" + 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_DEFAULT = (1024 * 1024).toString + val RETAINED_FILES_PROPERTY = "spark.executor.logs.rolling.maxRetainedFiles" + val DEFAULT_BUFFER_SIZE = 8192 + + /** + * Get the sorted list of rolled over files. This assumes that the all the rolled + * over file names are prefixed with the `activeFileName`, and the active file + * name has the latest logs. So it sorts all the rolled over logs (that are + * prefixed with `activeFileName`) and appends the active file + */ + def getSortedRolledOverFiles(directory: String, activeFileName: String): Seq[File] = { + val rolledOverFiles = new File(directory).getAbsoluteFile.listFiles.filter { file => + val fileName = file.getName + fileName.startsWith(activeFileName) && fileName != activeFileName + }.sorted + val activeFile = { + val file = new File(directory, activeFileName).getAbsoluteFile + if (file.exists) Some(file) else None + } + rolledOverFiles ++ activeFile + } +} diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala new file mode 100644 index 0000000000000..84e5c3c917dcb --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.logging + +import java.text.SimpleDateFormat +import java.util.Calendar + +import org.apache.spark.Logging + +/** + * Defines the policy based on which [[org.apache.spark.util.logging.RollingFileAppender]] will + * generate rolling files. + */ +private[spark] trait RollingPolicy { + + /** Whether rollover should be initiated at this moment */ + def shouldRollover(bytesToBeWritten: Long): Boolean + + /** Notify that rollover has occurred */ + def rolledOver() + + /** Notify that bytes have been written */ + def bytesWritten(bytes: Long) + + /** Get the desired name of the rollover file */ + def generateRolledOverFileSuffix(): String +} + +/** + * Defines a [[org.apache.spark.util.logging.RollingPolicy]] by which files will be rolled + * over at a fixed interval. + */ +private[spark] class TimeBasedRollingPolicy( + var rolloverIntervalMillis: Long, + rollingFileSuffixPattern: String, + checkIntervalConstraint: Boolean = true // set to false while testing + ) extends RollingPolicy with Logging { + + import TimeBasedRollingPolicy._ + if (checkIntervalConstraint && rolloverIntervalMillis < MINIMUM_INTERVAL_SECONDS * 1000L) { + logWarning(s"Rolling interval [${rolloverIntervalMillis/1000L} seconds] is too small. " + + s"Setting the interval to the acceptable minimum of $MINIMUM_INTERVAL_SECONDS seconds.") + rolloverIntervalMillis = MINIMUM_INTERVAL_SECONDS * 1000L + } + + @volatile private var nextRolloverTime = calculateNextRolloverTime() + private val formatter = new SimpleDateFormat(rollingFileSuffixPattern) + + /** Should rollover if current time has exceeded next rollover time */ + def shouldRollover(bytesToBeWritten: Long): Boolean = { + System.currentTimeMillis > nextRolloverTime + } + + /** Rollover has occurred, so find the next time to rollover */ + def rolledOver() { + nextRolloverTime = calculateNextRolloverTime() + logDebug(s"Current time: ${System.currentTimeMillis}, next rollover time: " + nextRolloverTime) + } + + def bytesWritten(bytes: Long) { } // nothing to do + + private def calculateNextRolloverTime(): Long = { + val now = System.currentTimeMillis() + val targetTime = ( + math.ceil(now.toDouble / rolloverIntervalMillis) * rolloverIntervalMillis + ).toLong + logDebug(s"Next rollover time is $targetTime") + targetTime + } + + def generateRolledOverFileSuffix(): String = { + formatter.format(Calendar.getInstance.getTime) + } +} + +private[spark] object TimeBasedRollingPolicy { + val MINIMUM_INTERVAL_SECONDS = 60L // 1 minute +} + +/** + * Defines a [[org.apache.spark.util.logging.RollingPolicy]] by which files will be rolled + * over after reaching a particular size. + */ +private[spark] class SizeBasedRollingPolicy( + var rolloverSizeBytes: Long, + checkSizeConstraint: Boolean = true // set to false while testing + ) extends RollingPolicy with Logging { + + import SizeBasedRollingPolicy._ + if (checkSizeConstraint && rolloverSizeBytes < MINIMUM_SIZE_BYTES) { + logWarning(s"Rolling size [$rolloverSizeBytes bytes] is too small. " + + s"Setting the size to the acceptable minimum of $MINIMUM_SIZE_BYTES bytes.") + rolloverSizeBytes = MINIMUM_SIZE_BYTES + } + + @volatile private var bytesWrittenSinceRollover = 0L + val formatter = new SimpleDateFormat("--YYYY-MM-dd--HH-mm-ss--SSSS") + + /** Should rollover if the next set of bytes is going to exceed the size limit */ + def shouldRollover(bytesToBeWritten: Long): Boolean = { + logInfo(s"$bytesToBeWritten + $bytesWrittenSinceRollover > $rolloverSizeBytes") + bytesToBeWritten + bytesWrittenSinceRollover > rolloverSizeBytes + } + + /** Rollover has occurred, so reset the counter */ + def rolledOver() { + bytesWrittenSinceRollover = 0 + } + + /** Increment the bytes that have been written in the current file */ + def bytesWritten(bytes: Long) { + bytesWrittenSinceRollover += bytes + } + + /** Get the desired name of the rollover file */ + def generateRolledOverFileSuffix(): String = { + formatter.format(Calendar.getInstance.getTime) + } +} + +private[spark] object SizeBasedRollingPolicy { + val MINIMUM_SIZE_BYTES = RollingFileAppender.DEFAULT_BUFFER_SIZE * 10 +} + diff --git a/core/src/main/scala/org/apache/spark/util/package-info.java b/core/src/main/scala/org/apache/spark/util/package-info.java new file mode 100644 index 0000000000000..819f54ee41a79 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/package-info.java @@ -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. + */ + +/** + * Spark utilities. + */ +package org.apache.spark.util; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/util/package.scala b/core/src/main/scala/org/apache/spark/util/package.scala new file mode 100644 index 0000000000000..c7099b85d225f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Spark utilities. + */ +package object util diff --git a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala index 98569143ee1e3..70f3dd62b9b19 100644 --- a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala @@ -17,9 +17,13 @@ package org.apache.spark.util.random +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A class with pseudorandom behavior. */ +@DeveloperApi trait Pseudorandom { /** Set random seed. */ def setSeed(seed: Long) 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 0f1fca4813ba9..247f10173f1e9 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 @@ -22,7 +22,10 @@ import java.util.Random import cern.jet.random.Poisson import cern.jet.random.engine.DRand +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A pseudorandom sampler. It is possible to change the sampled item type. For example, we might * want to add weights for stratified sampling or importance sampling. Should only use * transformations that are tied to the sampler and cannot be applied after sampling. @@ -30,6 +33,7 @@ import cern.jet.random.engine.DRand * @tparam T item type * @tparam U sampled item type */ +@DeveloperApi trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable { /** take a random sample */ @@ -40,6 +44,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable } /** + * :: DeveloperApi :: * A sampler based on Bernoulli trials. * * @param lb lower bound of the acceptance range @@ -47,6 +52,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable * @param complement whether to use the complement of the range specified, default to false * @tparam T item type */ +@DeveloperApi class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) (implicit random: Random = new XORShiftRandom) extends RandomSampler[T, T] { @@ -63,15 +69,22 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) } } - override def clone = new BernoulliSampler[T](lb, ub) + /** + * Return a sampler that is the complement of the range specified of the current sampler. + */ + def cloneComplement(): BernoulliSampler[T] = new BernoulliSampler[T](lb, ub, !complement) + + override def clone = new BernoulliSampler[T](lb, ub, complement) } /** + * :: DeveloperApi :: * A sampler based on values drawn from Poisson distribution. * * @param poisson a Poisson random number generator * @tparam T item type */ +@DeveloperApi class PoissonSampler[T](mean: Double) (implicit var poisson: Poisson = new Poisson(mean, new DRand)) extends RandomSampler[T, T] { diff --git a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala new file mode 100644 index 0000000000000..a79e3ee756fc6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.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.util.random + +private[spark] object SamplingUtils { + + /** + * Returns a sampling rate that guarantees a sample of size >= sampleSizeLowerBound 99.99% of + * the time. + * + * How the sampling rate is determined: + * Let p = num / total, where num is the sample size and total is the total number of + * datapoints in the RDD. We're trying to compute q > p such that + * - when sampling with replacement, we're drawing each datapoint with prob_i ~ Pois(q), + * where we want to guarantee Pr[s < num] < 0.0001 for s = sum(prob_i for i from 0 to total), + * i.e. the failure rate of not having a sufficiently large sample < 0.0001. + * Setting q = p + 5 * sqrt(p/total) is sufficient to guarantee 0.9999 success rate for + * num > 12, but we need a slightly larger q (9 empirically determined). + * - when sampling without replacement, we're drawing each datapoint with prob_i + * ~ Binomial(total, fraction) and our choice of q guarantees 1-delta, or 0.9999 success + * rate, where success rate is defined the same as in sampling with replacement. + * + * @param sampleSizeLowerBound sample size + * @param total size of RDD + * @param withReplacement whether sampling with replacement + * @return a sampling rate that guarantees sufficient sample size with 99.99% success rate + */ + def computeFractionForSampleSize(sampleSizeLowerBound: Int, total: Long, + withReplacement: Boolean): Double = { + val fraction = sampleSizeLowerBound.toDouble / total + if (withReplacement) { + val numStDev = if (sampleSizeLowerBound < 12) 9 else 5 + fraction + numStDev * math.sqrt(fraction / total) + } else { + val delta = 1e-4 + val gamma = - math.log(delta) / total + math.min(1, fraction + gamma + math.sqrt(gamma * gamma + 2 * gamma * fraction)) + } + } +} 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 8a4cdea2fa7b1..55b5713706178 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 @@ -25,28 +25,28 @@ import scala.util.hashing.MurmurHash3 import org.apache.spark.util.Utils.timeIt /** - * This class implements a XORShift random number generator algorithm + * This class implements a XORShift random number generator algorithm * Source: * Marsaglia, G. (2003). Xorshift RNGs. Journal of Statistical Software, Vol. 8, Issue 14. * @see Paper * This implementation is approximately 3.5 times faster than * {@link java.util.Random java.util.Random}, partly because of the algorithm, but also due - * to renouncing thread safety. JDK's implementation uses an AtomicLong seed, this class + * to renouncing thread safety. JDK's implementation uses an AtomicLong seed, this class * uses a regular Long. We can forgo thread safety since we use a new instance of the RNG * for each thread. */ private[spark] class XORShiftRandom(init: Long) extends JavaRandom(init) { - + def this() = this(System.nanoTime) private var seed = XORShiftRandom.hashSeed(init) // we need to just override next - this will be called by nextInt, nextDouble, // nextGaussian, nextLong, etc. - override protected def next(bits: Int): Int = { + override protected def next(bits: Int): Int = { var nextSeed = seed ^ (seed << 21) nextSeed ^= (nextSeed >>> 35) - nextSeed ^= (nextSeed << 4) + nextSeed ^= (nextSeed << 4) seed = nextSeed (nextSeed & ((1L << bits) -1)).asInstanceOf[Int] } @@ -89,17 +89,17 @@ private[spark] object XORShiftRandom { val million = 1e6.toInt val javaRand = new JavaRandom(seed) val xorRand = new XORShiftRandom(seed) - + // this is just to warm up the JIT - we're not timing anything - timeIt(1e6.toInt) { + timeIt(million) { javaRand.nextInt() xorRand.nextInt() } val iters = timeIt(numIters)(_) - + /* Return results as a map instead of just printing to screen - in case the user wants to do something with them */ + in case the user wants to do something with them */ Map("javaTime" -> iters {javaRand.nextInt()}, "xorTime" -> iters {xorRand.nextInt()}) diff --git a/core/src/main/scala/org/apache/spark/util/random/package-info.java b/core/src/main/scala/org/apache/spark/util/random/package-info.java new file mode 100644 index 0000000000000..62c3762dd11b6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/random/package-info.java @@ -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. + */ + +/** + * Utilities for random number generation. + */ +package org.apache.spark.util.random; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/util/random/package.scala b/core/src/main/scala/org/apache/spark/util/random/package.scala new file mode 100644 index 0000000000000..f361125fe2e8a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/random/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 + +/** + * Utilities for random number generation. + */ +package object random diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 2372f2d9924a1..b2868b59ce6c6 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -18,10 +18,17 @@ package org.apache.spark; import java.io.*; +import java.net.URI; import java.util.*; import scala.Tuple2; +import scala.Tuple3; +import scala.Tuple4; + +import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; import com.google.common.base.Optional; import com.google.common.base.Charsets; import com.google.common.io.Files; @@ -52,28 +59,19 @@ // see http://stackoverflow.com/questions/758570/. public class JavaAPISuite implements Serializable { private transient JavaSparkContext sc; + private transient File tempDir; @Before public void setUp() { sc = new JavaSparkContext("local", "JavaAPISuite"); + tempDir = Files.createTempDir(); + tempDir.deleteOnExit(); } @After public void tearDown() { sc.stop(); sc = null; - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.driver.port"); - } - - static class ReverseIntComparator implements Comparator, Serializable { - - @Override - public int compare(Integer a, Integer b) { - if (a > b) return -1; - else if (a < b) return 1; - else return 0; - } } @SuppressWarnings("unchecked") @@ -120,7 +118,7 @@ public void intersection() { JavaRDD intersections = s1.intersection(s2); Assert.assertEquals(3, intersections.count()); - ArrayList list = new ArrayList(); + List list = new ArrayList(); JavaRDD empty = sc.parallelize(list); JavaRDD emptyIntersection = empty.intersection(s2); Assert.assertEquals(0, emptyIntersection.count()); @@ -140,6 +138,28 @@ public void intersection() { Assert.assertEquals(2, pIntersection.count()); } + @Test + public void sample() { + List ints = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + JavaRDD rdd = sc.parallelize(ints); + JavaRDD sample20 = rdd.sample(true, 0.2, 11); + // expected 2 but of course result varies randomly a bit + Assert.assertEquals(3, sample20.count()); + JavaRDD sample20NoReplacement = rdd.sample(false, 0.2, 11); + Assert.assertEquals(2, sample20NoReplacement.count()); + } + + @Test + 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); + Assert.assertEquals(3, splits.length); + Assert.assertEquals(2, splits[0].count()); + Assert.assertEquals(3, splits[1].count()); + Assert.assertEquals(5, splits[2].count()); + } + @Test public void sortByKey() { List> pairs = new ArrayList>(); @@ -157,26 +177,82 @@ public void sortByKey() { Assert.assertEquals(new Tuple2(3, 2), sortedPairs.get(2)); // Custom comparator - sortedRDD = rdd.sortByKey(new ReverseIntComparator(), false); + sortedRDD = rdd.sortByKey(Collections.reverseOrder(), false); Assert.assertEquals(new Tuple2(-1, 1), sortedRDD.first()); sortedPairs = sortedRDD.collect(); Assert.assertEquals(new Tuple2(0, 4), sortedPairs.get(1)); Assert.assertEquals(new Tuple2(3, 2), sortedPairs.get(2)); } - static int foreachCalls = 0; + @Test + public void sortBy() { + List> pairs = new ArrayList>(); + pairs.add(new Tuple2(0, 4)); + pairs.add(new Tuple2(3, 2)); + pairs.add(new Tuple2(-1, 1)); + + JavaRDD> rdd = sc.parallelize(pairs); + + // compare on first value + JavaRDD> sortedRDD = rdd.sortBy(new Function, Integer>() { + public Integer call(Tuple2 t) throws Exception { + return t._1(); + } + }, true, 2); + + Assert.assertEquals(new Tuple2(-1, 1), sortedRDD.first()); + List> sortedPairs = sortedRDD.collect(); + Assert.assertEquals(new Tuple2(0, 4), sortedPairs.get(1)); + Assert.assertEquals(new Tuple2(3, 2), sortedPairs.get(2)); + + // compare on second value + sortedRDD = rdd.sortBy(new Function, Integer>() { + public Integer call(Tuple2 t) throws Exception { + return t._2(); + } + }, true, 2); + Assert.assertEquals(new Tuple2(-1, 1), sortedRDD.first()); + sortedPairs = sortedRDD.collect(); + Assert.assertEquals(new Tuple2(3, 2), sortedPairs.get(1)); + Assert.assertEquals(new Tuple2(0, 4), sortedPairs.get(2)); + } @Test public void foreach() { - foreachCalls = 0; + final Accumulator accum = sc.accumulator(0); JavaRDD rdd = sc.parallelize(Arrays.asList("Hello", "World")); rdd.foreach(new VoidFunction() { @Override - public void call(String s) { - foreachCalls++; + public void call(String s) throws IOException { + accum.add(1); } }); - Assert.assertEquals(2, foreachCalls); + Assert.assertEquals(2, accum.value().intValue()); + } + + @Test + public void toLocalIterator() { + List correct = Arrays.asList(1, 2, 3, 4); + JavaRDD rdd = sc.parallelize(correct); + List result = Lists.newArrayList(rdd.toLocalIterator()); + Assert.assertEquals(correct, result); + } + + @Test + public void zipWithUniqueId() { + List dataArray = Arrays.asList(1, 2, 3, 4); + JavaPairRDD zip = sc.parallelize(dataArray).zipWithUniqueId(); + JavaRDD indexes = zip.values(); + Assert.assertEquals(4, new HashSet(indexes.collect()).size()); + } + + @Test + public void zipWithIndex() { + List dataArray = Arrays.asList(1, 2, 3, 4); + JavaPairRDD zip = sc.parallelize(dataArray).zipWithIndex(); + JavaRDD indexes = zip.values(); + List correctIndexes = Arrays.asList(0L, 1L, 2L, 3L); + Assert.assertEquals(correctIndexes, indexes.collect()); } @SuppressWarnings("unchecked") @@ -188,7 +264,7 @@ public void lookup() { new Tuple2("Oranges", "Citrus") )); Assert.assertEquals(2, categories.lookup("Oranges").size()); - Assert.assertEquals(2, categories.groupByKey().lookup("Oranges").get(0).size()); + Assert.assertEquals(2, Iterables.size(categories.groupByKey().lookup("Oranges").get(0))); } @Test @@ -200,15 +276,15 @@ public Boolean call(Integer x) { return x % 2 == 0; } }; - JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); + JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size()); // Evens - Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds oddsAndEvens = rdd.groupBy(isOdd, 1); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size()); // Evens - Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds } @SuppressWarnings("unchecked") @@ -223,9 +299,71 @@ public void cogroup() { new Tuple2("Oranges", 2), new Tuple2("Apples", 3) )); - JavaPairRDD, List>> cogrouped = categories.cogroup(prices); - Assert.assertEquals("[Fruit, Citrus]", cogrouped.lookup("Oranges").get(0)._1().toString()); - Assert.assertEquals("[2]", cogrouped.lookup("Oranges").get(0)._2().toString()); + JavaPairRDD, Iterable>> cogrouped = + categories.cogroup(prices); + Assert.assertEquals("[Fruit, Citrus]", + Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); + Assert.assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); + + cogrouped.collect(); + } + + @SuppressWarnings("unchecked") + @Test + public void cogroup3() { + JavaPairRDD categories = sc.parallelizePairs(Arrays.asList( + new Tuple2("Apples", "Fruit"), + new Tuple2("Oranges", "Fruit"), + new Tuple2("Oranges", "Citrus") + )); + JavaPairRDD prices = sc.parallelizePairs(Arrays.asList( + new Tuple2("Oranges", 2), + new Tuple2("Apples", 3) + )); + JavaPairRDD quantities = sc.parallelizePairs(Arrays.asList( + new Tuple2("Oranges", 21), + new Tuple2("Apples", 42) + )); + + JavaPairRDD, Iterable, Iterable>> cogrouped = + categories.cogroup(prices, quantities); + Assert.assertEquals("[Fruit, Citrus]", + Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); + Assert.assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); + Assert.assertEquals("[42]", Iterables.toString(cogrouped.lookup("Apples").get(0)._3())); + + + cogrouped.collect(); + } + + @SuppressWarnings("unchecked") + @Test + public void cogroup4() { + JavaPairRDD categories = sc.parallelizePairs(Arrays.asList( + new Tuple2("Apples", "Fruit"), + new Tuple2("Oranges", "Fruit"), + new Tuple2("Oranges", "Citrus") + )); + JavaPairRDD prices = sc.parallelizePairs(Arrays.asList( + new Tuple2("Oranges", 2), + new Tuple2("Apples", 3) + )); + JavaPairRDD quantities = sc.parallelizePairs(Arrays.asList( + new Tuple2("Oranges", 21), + new Tuple2("Apples", 42) + )); + JavaPairRDD countries = sc.parallelizePairs(Arrays.asList( + new Tuple2("Oranges", "BR"), + new Tuple2("Apples", "US") + )); + + JavaPairRDD, Iterable, Iterable, Iterable>> cogrouped = + categories.cogroup(prices, quantities, countries); + Assert.assertEquals("[Fruit, Citrus]", + Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); + Assert.assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); + Assert.assertEquals("[42]", Iterables.toString(cogrouped.lookup("Apples").get(0)._3())); + Assert.assertEquals("[BR]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._4())); cogrouped.collect(); } @@ -252,8 +390,7 @@ public void leftOuterJoin() { rdd1.leftOuterJoin(rdd2).filter( new Function>>, Boolean>() { @Override - public Boolean call(Tuple2>> tup) - throws Exception { + public Boolean call(Tuple2>> tup) { return !tup._2()._2().isPresent(); } }).first(); @@ -277,6 +414,37 @@ public Integer call(Integer a, Integer b) { Assert.assertEquals(33, sum); } + @Test + public void aggregateByKey() { + JavaPairRDD pairs = sc.parallelizePairs( + Arrays.asList( + new Tuple2(1, 1), + new Tuple2(1, 1), + new Tuple2(3, 2), + new Tuple2(5, 1), + new Tuple2(5, 3)), 2); + + Map> sets = pairs.aggregateByKey(new HashSet(), + new Function2, Integer, Set>() { + @Override + public Set call(Set a, Integer b) { + a.add(b); + return a; + } + }, + new Function2, Set, Set>() { + @Override + public Set call(Set a, Set b) { + a.addAll(b); + return a; + } + }).collectAsMap(); + Assert.assertEquals(3, sets.size()); + Assert.assertEquals(new HashSet(Arrays.asList(1)), sets.get(1)); + Assert.assertEquals(new HashSet(Arrays.asList(2)), sets.get(3)); + Assert.assertEquals(new HashSet(Arrays.asList(1, 3)), sets.get(5)); + } + @SuppressWarnings("unchecked") @Test public void foldByKey() { @@ -327,8 +495,7 @@ public Integer call(Integer a, Integer b) { Assert.assertEquals(2, localCounts.get(2).intValue()); Assert.assertEquals(3, localCounts.get(3).intValue()); - localCounts = rdd.reduceByKeyLocally(new Function2() { + localCounts = rdd.reduceByKeyLocally(new Function2() { @Override public Integer call(Integer a, Integer b) { return a + b; @@ -419,16 +586,17 @@ public void map() { JavaDoubleRDD doubles = rdd.mapToDouble(new DoubleFunction() { @Override public double call(Integer x) { - return 1.0 * x; + return x.doubleValue(); } }).cache(); doubles.collect(); - JavaPairRDD pairs = rdd.mapToPair(new PairFunction() { - @Override - public Tuple2 call(Integer x) { - return new Tuple2(x, x); - } - }).cache(); + JavaPairRDD pairs = rdd.mapToPair( + new PairFunction() { + @Override + public Tuple2 call(Integer x) { + return new Tuple2(x, x); + } + }).cache(); pairs.collect(); JavaRDD strings = rdd.map(new Function() { @Override @@ -458,7 +626,9 @@ public Iterable call(String x) { @Override public Iterable> call(String s) { List> pairs = new LinkedList>(); - for (String word : s.split(" ")) pairs.add(new Tuple2(word, word)); + for (String word : s.split(" ")) { + pairs.add(new Tuple2(word, word)); + } return pairs; } } @@ -470,7 +640,9 @@ public Iterable> call(String s) { @Override public Iterable call(String s) { List lengths = new LinkedList(); - for (String word : s.split(" ")) lengths.add(word.length() * 1.0); + for (String word : s.split(" ")) { + lengths.add((double) word.length()); + } return lengths; } }); @@ -492,7 +664,7 @@ public void mapsFromPairsToPairs() { JavaPairRDD swapped = pairRDD.flatMapToPair( new PairFlatMapFunction, String, Integer>() { @Override - public Iterable> call(Tuple2 item) throws Exception { + public Iterable> call(Tuple2 item) { return Collections.singletonList(item.swap()); } }); @@ -501,7 +673,7 @@ public Iterable> call(Tuple2 item) thro // There was never a bug here, but it's worth testing: pairRDD.mapToPair(new PairFunction, String, Integer>() { @Override - public Tuple2 call(Tuple2 item) throws Exception { + public Tuple2 call(Tuple2 item) { return item.swap(); } }).collect(); @@ -569,8 +741,8 @@ public void persist() { @Test public void iterator() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); - TaskContext context = new TaskContext(0, 0, 0, false, false, new TaskMetrics()); - Assert.assertEquals(1, rdd.iterator(rdd.splits().get(0), context).next().intValue()); + TaskContext context = new TaskContext(0, 0, 0, false, new TaskMetrics()); + Assert.assertEquals(1, rdd.iterator(rdd.partitions().get(0), context).next().intValue()); } @Test @@ -583,7 +755,6 @@ public void glom() { @Test public void textFiles() throws IOException { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); rdd.saveAsTextFile(outputDir); @@ -598,34 +769,28 @@ public void textFiles() throws IOException { } @Test - public void wholeTextFiles() throws IOException { - byte[] content1 = "spark is easy to use.\n".getBytes(); - byte[] content2 = "spark is also easy to use.\n".getBytes(); + public void wholeTextFiles() throws Exception { + byte[] content1 = "spark is easy to use.\n".getBytes("utf-8"); + byte[] content2 = "spark is also easy to use.\n".getBytes("utf-8"); - File tempDir = Files.createTempDir(); String tempDirName = tempDir.getAbsolutePath(); - DataOutputStream ds = new DataOutputStream(new FileOutputStream(tempDirName + "/part-00000")); - ds.write(content1); - ds.close(); - ds = new DataOutputStream(new FileOutputStream(tempDirName + "/part-00001")); - ds.write(content2); - ds.close(); - - HashMap container = new HashMap(); + Files.write(content1, new File(tempDirName + "/part-00000")); + Files.write(content2, new File(tempDirName + "/part-00001")); + + Map container = new HashMap(); container.put(tempDirName+"/part-00000", new Text(content1).toString()); container.put(tempDirName+"/part-00001", new Text(content2).toString()); - JavaPairRDD readRDD = sc.wholeTextFiles(tempDirName); + JavaPairRDD readRDD = sc.wholeTextFiles(tempDirName, 3); List> result = readRDD.collect(); for (Tuple2 res : result) { - Assert.assertEquals(res._2(), container.get(res._1())); + Assert.assertEquals(res._2(), container.get(new URI(res._1()).getPath())); } } @Test public void textFilesCompressed() throws IOException { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); rdd.saveAsTextFile(outputDir, DefaultCodec.class); @@ -639,7 +804,6 @@ public void textFilesCompressed() throws IOException { @SuppressWarnings("unchecked") @Test public void sequenceFile() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -669,7 +833,6 @@ public Tuple2 call(Tuple2 pair) { @SuppressWarnings("unchecked") @Test public void writeWithNewAPIHadoopFile() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -700,7 +863,6 @@ public String call(Tuple2 x) { @SuppressWarnings("unchecked") @Test public void readWithNewAPIHadoopFile() throws IOException { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -730,7 +892,6 @@ public String call(Tuple2 x) { @Test public void objectFilesOfInts() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); rdd.saveAsObjectFile(outputDir); @@ -743,7 +904,6 @@ public void objectFilesOfInts() { @SuppressWarnings("unchecked") @Test public void objectFilesOfComplexTypes() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -760,7 +920,6 @@ public void objectFilesOfComplexTypes() { @SuppressWarnings("unchecked") @Test public void hadoopFile() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -790,7 +949,6 @@ public String call(Tuple2 x) { @SuppressWarnings("unchecked") @Test public void hadoopFileCompressed() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output_compressed").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -825,7 +983,7 @@ public void zip() { JavaDoubleRDD doubles = rdd.mapToDouble(new DoubleFunction() { @Override public double call(Integer x) { - return 1.0 * x; + return x.doubleValue(); } }); JavaPairRDD zipped = rdd.zip(doubles); @@ -840,17 +998,7 @@ public void zipPartitions() { new FlatMapFunction2, Iterator, Integer>() { @Override public Iterable call(Iterator i, Iterator s) { - int sizeI = 0; - int sizeS = 0; - while (i.hasNext()) { - sizeI += 1; - i.next(); - } - while (s.hasNext()) { - sizeS += 1; - s.next(); - } - return Arrays.asList(sizeI, sizeS); + return Arrays.asList(Iterators.size(i), Iterators.size(s)); } }; @@ -864,6 +1012,7 @@ public void accumulators() { final Accumulator intAccum = sc.intAccumulator(10); rdd.foreach(new VoidFunction() { + @Override public void call(Integer x) { intAccum.add(x); } @@ -872,6 +1021,7 @@ public void call(Integer x) { final Accumulator doubleAccum = sc.doubleAccumulator(10.0); rdd.foreach(new VoidFunction() { + @Override public void call(Integer x) { doubleAccum.add((double) x); } @@ -880,14 +1030,17 @@ public void call(Integer x) { // Try a custom accumulator type AccumulatorParam floatAccumulatorParam = new AccumulatorParam() { + @Override public Float addInPlace(Float r, Float t) { return r + t; } + @Override public Float addAccumulator(Float r, Float t) { return r + t; } + @Override public Float zero(Float initialValue) { return 0.0f; } @@ -895,6 +1048,7 @@ public Float zero(Float initialValue) { final Accumulator floatAccum = sc.accumulator(10.0f, floatAccumulatorParam); rdd.foreach(new VoidFunction() { + @Override public void call(Integer x) { floatAccum.add((float) x); } @@ -910,7 +1064,8 @@ public void call(Integer x) { public void keyBy() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2)); List> s = rdd.keyBy(new Function() { - public String call(Integer t) throws Exception { + @Override + public String call(Integer t) { return t.toString(); } }).collect(); @@ -920,25 +1075,23 @@ public String call(Integer t) throws Exception { @Test public void checkpointAndComputation() { - File tempDir = Files.createTempDir(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); sc.setCheckpointDir(tempDir.getAbsolutePath()); - Assert.assertEquals(false, rdd.isCheckpointed()); + Assert.assertFalse(rdd.isCheckpointed()); rdd.checkpoint(); rdd.count(); // Forces the DAG to cause a checkpoint - Assert.assertEquals(true, rdd.isCheckpointed()); + Assert.assertTrue(rdd.isCheckpointed()); Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), rdd.collect()); } @Test public void checkpointAndRestore() { - File tempDir = Files.createTempDir(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); sc.setCheckpointDir(tempDir.getAbsolutePath()); - Assert.assertEquals(false, rdd.isCheckpointed()); + Assert.assertFalse(rdd.isCheckpointed()); rdd.checkpoint(); rdd.count(); // Forces the DAG to cause a checkpoint - Assert.assertEquals(true, rdd.isCheckpointed()); + Assert.assertTrue(rdd.isCheckpointed()); Assert.assertTrue(rdd.getCheckpointFile().isPresent()); JavaRDD recovered = sc.checkpointFile(rdd.getCheckpointFile().get()); @@ -949,16 +1102,17 @@ public void checkpointAndRestore() { @Test public void mapOnPairRDD() { JavaRDD rdd1 = sc.parallelize(Arrays.asList(1,2,3,4)); - JavaPairRDD rdd2 = rdd1.mapToPair(new PairFunction() { - @Override - public Tuple2 call(Integer i) throws Exception { - return new Tuple2(i, i % 2); - } - }); + JavaPairRDD rdd2 = rdd1.mapToPair( + new PairFunction() { + @Override + public Tuple2 call(Integer i) { + return new Tuple2(i, i % 2); + } + }); JavaPairRDD rdd3 = rdd2.mapToPair( new PairFunction, Integer, Integer>() { @Override - public Tuple2 call(Tuple2 in) throws Exception { + public Tuple2 call(Tuple2 in) { return new Tuple2(in._2(), in._1()); } }); @@ -975,14 +1129,15 @@ public Tuple2 call(Tuple2 in) throws Excepti public void collectPartitions() { JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3); - JavaPairRDD rdd2 = rdd1.mapToPair(new PairFunction() { - @Override - public Tuple2 call(Integer i) throws Exception { - return new Tuple2(i, i % 2); - } - }); + JavaPairRDD rdd2 = rdd1.mapToPair( + new PairFunction() { + @Override + public Tuple2 call(Integer i) { + return new Tuple2(i, i % 2); + } + }); - List[] parts = rdd1.collectPartitions(new int[] {0}); + List[] parts = rdd1.collectPartitions(new int[] {0}); Assert.assertEquals(Arrays.asList(1, 2), parts[0]); parts = rdd1.collectPartitions(new int[] {1, 2}); @@ -993,14 +1148,14 @@ public Tuple2 call(Integer i) throws Exception { new Tuple2(2, 0)), rdd2.collectPartitions(new int[] {0})[0]); - parts = rdd2.collectPartitions(new int[] {1, 2}); + List>[] parts2 = rdd2.collectPartitions(new int[] {1, 2}); Assert.assertEquals(Arrays.asList(new Tuple2(3, 1), new Tuple2(4, 0)), - parts[0]); + parts2[0]); Assert.assertEquals(Arrays.asList(new Tuple2(5, 1), new Tuple2(6, 0), new Tuple2(7, 1)), - parts[1]); + parts2[1]); } @Test @@ -1011,27 +1166,25 @@ public void countApproxDistinct() { arrayData.add(i % size); } JavaRDD simpleRdd = sc.parallelize(arrayData, 10); - Assert.assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.2) - size) / (size * 1.0)) < 0.2); - Assert.assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.05) - size) / (size * 1.0)) <= 0.05); - Assert.assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.01) - size) / (size * 1.0)) <= 0.01); + Assert.assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.05) - size) / (size * 1.0)) <= 0.1); } @Test public void countApproxDistinctByKey() { - double relativeSD = 0.001; - List> arrayData = new ArrayList>(); - for (int i = 10; i < 100; i++) - for (int j = 0; j < i; j++) + for (int i = 10; i < 100; i++) { + for (int j = 0; j < i; j++) { arrayData.add(new Tuple2(i, j)); - + } + } + double relativeSD = 0.001; JavaPairRDD pairRdd = sc.parallelizePairs(arrayData); - List> res = pairRdd.countApproxDistinctByKey(relativeSD).collect(); + List> res = pairRdd.countApproxDistinctByKey(8, 0).collect(); for (Tuple2 resItem : res) { double count = (double)resItem._1(); Long resCount = (Long)resItem._2(); Double error = Math.abs((resCount - count) / count); - Assert.assertTrue(error < relativeSD); + Assert.assertTrue(error < 0.1); } } @@ -1040,12 +1193,13 @@ public void countApproxDistinctByKey() { public void collectAsMapWithIntArrayValues() { // Regression test for SPARK-1040 JavaRDD rdd = sc.parallelize(Arrays.asList(1)); - JavaPairRDD pairRDD = rdd.mapToPair(new PairFunction() { - @Override - public Tuple2 call(Integer x) throws Exception { - return new Tuple2(x, new int[] { x }); - } - }); + JavaPairRDD pairRDD = rdd.mapToPair( + new PairFunction() { + @Override + public Tuple2 call(Integer x) { + return new Tuple2(x, new int[] { x }); + } + }); pairRDD.collect(); // Works fine pairRDD.collectAsMap(); // Used to crash with ClassCastException } diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index f6fef03689a7c..26b73a1b39744 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=core/target/unit-tests.log +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} %p %c{1}: %m%n diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 4e7c34e6d1ada..3aab88e9e9196 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -20,11 +20,11 @@ package org.apache.spark import scala.collection.mutable import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers +import org.scalatest.Matchers import org.apache.spark.SparkContext._ -class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkContext { +class AccumulatorSuite extends FunSuite with Matchers with LocalSparkContext { implicit def setAccum[A] = new AccumulableParam[mutable.Set[A], A] { diff --git a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala index d2e303d81c4c8..4ab870e751778 100644 --- a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala @@ -37,9 +37,8 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val securityManager = new SecurityManager(conf); val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, securityManager = securityManager) - System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext System.setProperty("spark.hostPort", hostname + ":" + boundPort) assert(securityManager.isAuthenticationEnabled() === true) @@ -54,14 +53,14 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { assert(securityManagerBad.isAuthenticationEnabled() === true) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf, securityManager = securityManagerBad) - val slaveTracker = new MapOutputTracker(conf) + val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) - intercept[akka.actor.ActorNotFound] { - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + intercept[akka.actor.ActorNotFound] { + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) } actorSystem.shutdown() @@ -75,9 +74,8 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val securityManager = new SecurityManager(conf); val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, securityManager = securityManager) - System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext System.setProperty("spark.hostPort", hostname + ":" + boundPort) assert(securityManager.isAuthenticationEnabled() === false) @@ -91,9 +89,9 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { badconf.set("spark.authenticate.secret", "good") val securityManagerBad = new SecurityManager(badconf); - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = badconf, securityManager = securityManagerBad) - val slaveTracker = new MapOutputTracker(conf) + val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) @@ -127,9 +125,8 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val securityManager = new SecurityManager(conf); val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, securityManager = securityManager) - System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext System.setProperty("spark.hostPort", hostname + ":" + boundPort) assert(securityManager.isAuthenticationEnabled() === true) @@ -147,7 +144,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = goodconf, securityManager = securityManagerGood) - val slaveTracker = new MapOutputTracker(conf) + val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) @@ -180,9 +177,8 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val securityManager = new SecurityManager(conf); val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, securityManager = securityManager) - System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext System.setProperty("spark.hostPort", hostname + ":" + boundPort) assert(securityManager.isAuthenticationEnabled() === true) @@ -200,12 +196,12 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = badconf, securityManager = securityManagerBad) - val slaveTracker = new MapOutputTracker(conf) + val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) - intercept[akka.actor.ActorNotFound] { - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + intercept[akka.actor.ActorNotFound] { + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) } actorSystem.shutdown() diff --git a/core/src/test/scala/org/apache/spark/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/BroadcastSuite.scala index 96ba3929c1685..c9936256a5b95 100644 --- a/core/src/test/scala/org/apache/spark/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/BroadcastSuite.scala @@ -19,68 +19,297 @@ package org.apache.spark import org.scalatest.FunSuite -class BroadcastSuite extends FunSuite with LocalSparkContext { +import org.apache.spark.storage._ +import org.apache.spark.broadcast.{Broadcast, HttpBroadcast} +import org.apache.spark.storage.BroadcastBlockId +class BroadcastSuite extends FunSuite with LocalSparkContext { - override def afterEach() { - super.afterEach() - System.clearProperty("spark.broadcast.factory") - } + private val httpConf = broadcastConf("HttpBroadcastFactory") + private val torrentConf = broadcastConf("TorrentBroadcastFactory") test("Using HttpBroadcast locally") { - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") - sc = new SparkContext("local", "test") - val list = List(1, 2, 3, 4) - val listBroadcast = sc.broadcast(list) - val results = sc.parallelize(1 to 2).map(x => (x, listBroadcast.value.sum)) - assert(results.collect.toSet === Set((1, 10), (2, 10))) + sc = new SparkContext("local", "test", httpConf) + val list = List[Int](1, 2, 3, 4) + val broadcast = sc.broadcast(list) + val results = sc.parallelize(1 to 2).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === Set((1, 10), (2, 10))) } test("Accessing HttpBroadcast variables from multiple threads") { - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") - sc = new SparkContext("local[10]", "test") - val list = List(1, 2, 3, 4) - val listBroadcast = sc.broadcast(list) - val results = sc.parallelize(1 to 10).map(x => (x, listBroadcast.value.sum)) - assert(results.collect.toSet === (1 to 10).map(x => (x, 10)).toSet) + sc = new SparkContext("local[10]", "test", httpConf) + val list = List[Int](1, 2, 3, 4) + val broadcast = sc.broadcast(list) + val results = sc.parallelize(1 to 10).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === (1 to 10).map(x => (x, 10)).toSet) } test("Accessing HttpBroadcast variables in a local cluster") { - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") val numSlaves = 4 - sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test") - val list = List(1, 2, 3, 4) - val listBroadcast = sc.broadcast(list) - val results = sc.parallelize(1 to numSlaves).map(x => (x, listBroadcast.value.sum)) - assert(results.collect.toSet === (1 to numSlaves).map(x => (x, 10)).toSet) + sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", httpConf) + val list = List[Int](1, 2, 3, 4) + val broadcast = sc.broadcast(list) + val results = sc.parallelize(1 to numSlaves).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === (1 to numSlaves).map(x => (x, 10)).toSet) } test("Using TorrentBroadcast locally") { - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.TorrentBroadcastFactory") - sc = new SparkContext("local", "test") - val list = List(1, 2, 3, 4) - val listBroadcast = sc.broadcast(list) - val results = sc.parallelize(1 to 2).map(x => (x, listBroadcast.value.sum)) - assert(results.collect.toSet === Set((1, 10), (2, 10))) + sc = new SparkContext("local", "test", torrentConf) + val list = List[Int](1, 2, 3, 4) + val broadcast = sc.broadcast(list) + val results = sc.parallelize(1 to 2).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === Set((1, 10), (2, 10))) } test("Accessing TorrentBroadcast variables from multiple threads") { - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.TorrentBroadcastFactory") - sc = new SparkContext("local[10]", "test") - val list = List(1, 2, 3, 4) - val listBroadcast = sc.broadcast(list) - val results = sc.parallelize(1 to 10).map(x => (x, listBroadcast.value.sum)) - assert(results.collect.toSet === (1 to 10).map(x => (x, 10)).toSet) + sc = new SparkContext("local[10]", "test", torrentConf) + val list = List[Int](1, 2, 3, 4) + val broadcast = sc.broadcast(list) + val results = sc.parallelize(1 to 10).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === (1 to 10).map(x => (x, 10)).toSet) } test("Accessing TorrentBroadcast variables in a local cluster") { - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.TorrentBroadcastFactory") val numSlaves = 4 - sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test") - val list = List(1, 2, 3, 4) - val listBroadcast = sc.broadcast(list) - val results = sc.parallelize(1 to numSlaves).map(x => (x, listBroadcast.value.sum)) - assert(results.collect.toSet === (1 to numSlaves).map(x => (x, 10)).toSet) + sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", torrentConf) + val list = List[Int](1, 2, 3, 4) + val broadcast = sc.broadcast(list) + val results = sc.parallelize(1 to numSlaves).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === (1 to numSlaves).map(x => (x, 10)).toSet) + } + + test("Unpersisting HttpBroadcast on executors only in local mode") { + testUnpersistHttpBroadcast(distributed = false, removeFromDriver = false) + } + + test("Unpersisting HttpBroadcast on executors and driver in local mode") { + testUnpersistHttpBroadcast(distributed = false, removeFromDriver = true) + } + + test("Unpersisting HttpBroadcast on executors only in distributed mode") { + testUnpersistHttpBroadcast(distributed = true, removeFromDriver = false) + } + + test("Unpersisting HttpBroadcast on executors and driver in distributed mode") { + testUnpersistHttpBroadcast(distributed = true, removeFromDriver = true) + } + + test("Unpersisting TorrentBroadcast on executors only in local mode") { + testUnpersistTorrentBroadcast(distributed = false, removeFromDriver = false) + } + + test("Unpersisting TorrentBroadcast on executors and driver in local mode") { + testUnpersistTorrentBroadcast(distributed = false, removeFromDriver = true) + } + + test("Unpersisting TorrentBroadcast on executors only in distributed mode") { + testUnpersistTorrentBroadcast(distributed = true, removeFromDriver = false) + } + + test("Unpersisting TorrentBroadcast on executors and driver in distributed mode") { + testUnpersistTorrentBroadcast(distributed = true, removeFromDriver = true) + } + /** + * Verify the persistence of state associated with an HttpBroadcast in either local mode or + * local-cluster mode (when distributed = true). + * + * This test creates a broadcast variable, uses it on all executors, and then unpersists it. + * In between each step, this test verifies that the broadcast blocks and the broadcast file + * are present only on the expected nodes. + */ + private def testUnpersistHttpBroadcast(distributed: Boolean, removeFromDriver: Boolean) { + val numSlaves = if (distributed) 2 else 0 + + def getBlockIds(id: Long) = Seq[BroadcastBlockId](BroadcastBlockId(id)) + + // Verify that the broadcast file is created, and blocks are persisted only on the driver + def afterCreation(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { + assert(blockIds.size === 1) + val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true) + assert(statuses.size === 1) + statuses.head match { case (bm, status) => + assert(bm.executorId === "", "Block should only be on the driver") + assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) + assert(status.memSize > 0, "Block should be in memory store on the driver") + assert(status.diskSize === 0, "Block should not be in disk store on the driver") + } + if (distributed) { + // this file is only generated in distributed mode + assert(HttpBroadcast.getFile(blockIds.head.broadcastId).exists, "Broadcast file not found!") + } + } + + // Verify that blocks are persisted in both the executors and the driver + def afterUsingBroadcast(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { + assert(blockIds.size === 1) + val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true) + assert(statuses.size === numSlaves + 1) + statuses.foreach { case (_, status) => + assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) + assert(status.memSize > 0, "Block should be in memory store") + assert(status.diskSize === 0, "Block should not be in disk store") + } + } + + // Verify that blocks are unpersisted on all executors, and on all nodes if removeFromDriver + // is true. In the latter case, also verify that the broadcast file is deleted on the driver. + def afterUnpersist(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { + assert(blockIds.size === 1) + val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true) + val expectedNumBlocks = if (removeFromDriver) 0 else 1 + val possiblyNot = if (removeFromDriver) "" else " not" + assert(statuses.size === expectedNumBlocks, + "Block should%s be unpersisted on the driver".format(possiblyNot)) + if (distributed && removeFromDriver) { + // this file is only generated in distributed mode + assert(!HttpBroadcast.getFile(blockIds.head.broadcastId).exists, + "Broadcast file should%s be deleted".format(possiblyNot)) + } + } + + testUnpersistBroadcast(distributed, numSlaves, httpConf, getBlockIds, afterCreation, + afterUsingBroadcast, afterUnpersist, removeFromDriver) + } + + /** + * Verify the persistence of state associated with an TorrentBroadcast in a local-cluster. + * + * This test creates a broadcast variable, uses it on all executors, and then unpersists it. + * In between each step, this test verifies that the broadcast blocks are present only on the + * expected nodes. + */ + private def testUnpersistTorrentBroadcast(distributed: Boolean, removeFromDriver: Boolean) { + val numSlaves = if (distributed) 2 else 0 + + def getBlockIds(id: Long) = { + val broadcastBlockId = BroadcastBlockId(id) + val metaBlockId = BroadcastBlockId(id, "meta") + // Assume broadcast value is small enough to fit into 1 piece + val pieceBlockId = BroadcastBlockId(id, "piece0") + if (distributed) { + // the metadata and piece blocks are generated only in distributed mode + Seq[BroadcastBlockId](broadcastBlockId, metaBlockId, pieceBlockId) + } else { + Seq[BroadcastBlockId](broadcastBlockId) + } + } + + // Verify that blocks are persisted only on the driver + def afterCreation(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { + blockIds.foreach { blockId => + val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true) + assert(statuses.size === 1) + statuses.head match { case (bm, status) => + assert(bm.executorId === "", "Block should only be on the driver") + assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) + assert(status.memSize > 0, "Block should be in memory store on the driver") + assert(status.diskSize === 0, "Block should not be in disk store on the driver") + } + } + } + + // Verify that blocks are persisted in both the executors and the driver + def afterUsingBroadcast(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { + blockIds.foreach { blockId => + val statuses = bmm.getBlockStatus(blockId, askSlaves = true) + if (blockId.field == "meta") { + // Meta data is only on the driver + assert(statuses.size === 1) + statuses.head match { case (bm, _) => assert(bm.executorId === "") } + } else { + // Other blocks are on both the executors and the driver + assert(statuses.size === numSlaves + 1, + blockId + " has " + statuses.size + " statuses: " + statuses.mkString(",")) + statuses.foreach { case (_, status) => + assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) + assert(status.memSize > 0, "Block should be in memory store") + assert(status.diskSize === 0, "Block should not be in disk store") + } + } + } + } + + // Verify that blocks are unpersisted on all executors, and on all nodes if removeFromDriver + // is true. + def afterUnpersist(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { + val expectedNumBlocks = if (removeFromDriver) 0 else 1 + val possiblyNot = if (removeFromDriver) "" else " not" + blockIds.foreach { blockId => + val statuses = bmm.getBlockStatus(blockId, askSlaves = true) + assert(statuses.size === expectedNumBlocks, + "Block should%s be unpersisted on the driver".format(possiblyNot)) + } + } + + testUnpersistBroadcast(distributed, numSlaves, torrentConf, getBlockIds, afterCreation, + afterUsingBroadcast, afterUnpersist, removeFromDriver) + } + + /** + * This test runs in 4 steps: + * + * 1) Create broadcast variable, and verify that all state is persisted on the driver. + * 2) Use the broadcast variable on all executors, and verify that all state is persisted + * on both the driver and the executors. + * 3) Unpersist the broadcast, and verify that all state is removed where they should be. + * 4) [Optional] If removeFromDriver is false, we verify that the broadcast is re-usable. + */ + private def testUnpersistBroadcast( + distributed: Boolean, + numSlaves: Int, // used only when distributed = true + broadcastConf: SparkConf, + getBlockIds: Long => Seq[BroadcastBlockId], + afterCreation: (Seq[BroadcastBlockId], BlockManagerMaster) => Unit, + afterUsingBroadcast: (Seq[BroadcastBlockId], BlockManagerMaster) => Unit, + afterUnpersist: (Seq[BroadcastBlockId], BlockManagerMaster) => Unit, + removeFromDriver: Boolean) { + + sc = if (distributed) { + new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", broadcastConf) + } else { + new SparkContext("local", "test", broadcastConf) + } + val blockManagerMaster = sc.env.blockManager.master + val list = List[Int](1, 2, 3, 4) + + // Create broadcast variable + val broadcast = sc.broadcast(list) + val blocks = getBlockIds(broadcast.id) + afterCreation(blocks, blockManagerMaster) + + // Use broadcast variable on all executors + val partitions = 10 + assert(partitions > numSlaves) + val results = sc.parallelize(1 to partitions, partitions).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === (1 to partitions).map(x => (x, list.sum)).toSet) + afterUsingBroadcast(blocks, blockManagerMaster) + + // Unpersist broadcast + if (removeFromDriver) { + broadcast.destroy(blocking = true) + } else { + broadcast.unpersist(blocking = true) + } + afterUnpersist(blocks, blockManagerMaster) + + // If the broadcast is removed from driver, all subsequent uses of the broadcast variable + // should throw SparkExceptions. Otherwise, the result should be the same as before. + if (removeFromDriver) { + // Using this variable on the executors crashes them, which hangs the test. + // Instead, crash the driver by directly accessing the broadcast value. + intercept[SparkException] { broadcast.value } + intercept[SparkException] { broadcast.unpersist() } + intercept[SparkException] { broadcast.destroy(blocking = true) } + } else { + val results = sc.parallelize(1 to partitions, partitions).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === (1 to partitions).map(x => (x, list.sum)).toSet) + } } + /** Helper method to create a SparkConf that uses the given broadcast factory. */ + private def broadcastConf(factoryName: String): SparkConf = { + val conf = new SparkConf + conf.set("spark.broadcast.factory", "org.apache.spark.broadcast.%s".format(factoryName)) + conf + } } diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index b86923f07f793..4f178db40f638 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -23,7 +23,6 @@ import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.mock.EasyMockSugar import org.apache.spark.rdd.RDD -import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage._ // TODO: Test the CacheManager's thread-safety aspects @@ -59,8 +58,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } whenExecuting(blockManager) { - val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = TaskMetrics.empty()) + val context = new TaskContext(0, 0, 0) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } @@ -72,8 +70,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } whenExecuting(blockManager) { - val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = TaskMetrics.empty()) + val context = new TaskContext(0, 0, 0) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(5, 6, 7)) } @@ -86,8 +83,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } whenExecuting(blockManager) { - val context = new TaskContext(0, 0, 0, runningLocally = true, interrupted = false, - taskMetrics = TaskMetrics.empty()) + val context = new TaskContext(0, 0, 0, runningLocally = true) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index d2555b7c052c1..f64f3c9036034 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -35,6 +35,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { override def beforeEach() { super.beforeEach() checkpointDir = File.createTempFile("temp", "") + checkpointDir.deleteOnExit() checkpointDir.delete() sc = new SparkContext("local", "test") sc.setCheckpointDir(checkpointDir.toString) @@ -42,9 +43,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { override def afterEach() { super.afterEach() - if (checkpointDir != null) { - checkpointDir.delete() - } + Utils.deleteRecursively(checkpointDir) } test("basic checkpointing") { @@ -168,26 +167,28 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { }) } - test("ZippedRDD") { - testRDD(rdd => new ZippedRDD(sc, rdd, rdd.map(x => x))) - testRDDPartitions(rdd => new ZippedRDD(sc, rdd, rdd.map(x => x))) + test("ZippedPartitionsRDD") { + testRDD(rdd => rdd.zip(rdd.map(x => x))) + testRDDPartitions(rdd => rdd.zip(rdd.map(x => x))) - // Test that the ZippedPartition updates parent partitions - // after the parent RDD has been checkpointed and parent partitions have been changed. - // Note that this test is very specific to the current implementation of ZippedRDD. + // Test that ZippedPartitionsRDD updates parent partitions after parent RDDs have + // been checkpointed and parent partitions have been changed. + // Note that this test is very specific to the implementation of ZippedPartitionsRDD. val rdd = generateFatRDD() - val zippedRDD = new ZippedRDD(sc, rdd, rdd.map(x => x)) + val zippedRDD = rdd.zip(rdd.map(x => x)).asInstanceOf[ZippedPartitionsRDD2[_, _, _]] zippedRDD.rdd1.checkpoint() zippedRDD.rdd2.checkpoint() val partitionBeforeCheckpoint = - serializeDeserialize(zippedRDD.partitions.head.asInstanceOf[ZippedPartition[_, _]]) + serializeDeserialize(zippedRDD.partitions.head.asInstanceOf[ZippedPartitionsPartition]) zippedRDD.count() val partitionAfterCheckpoint = - serializeDeserialize(zippedRDD.partitions.head.asInstanceOf[ZippedPartition[_, _]]) + serializeDeserialize(zippedRDD.partitions.head.asInstanceOf[ZippedPartitionsPartition]) assert( - partitionAfterCheckpoint.partition1.getClass != partitionBeforeCheckpoint.partition1.getClass && - partitionAfterCheckpoint.partition2.getClass != partitionBeforeCheckpoint.partition2.getClass, - "ZippedRDD.partition1 and ZippedRDD.partition2 not updated after parent RDD is checkpointed" + partitionAfterCheckpoint.partitions(0).getClass != + partitionBeforeCheckpoint.partitions(0).getClass && + partitionAfterCheckpoint.partitions(1).getClass != + partitionBeforeCheckpoint.partitions(1).getClass, + "ZippedPartitionsRDD partition 0 (or 1) not updated after parent RDDs are checkpointed" ) } diff --git a/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala index 80f7ec00c74b2..df6b2604c8d8a 100644 --- a/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.network.{ConnectionManager, Message, ConnectionManagerId import scala.concurrent.Await import scala.concurrent.TimeoutException import scala.concurrent.duration._ - +import scala.language.postfixOps /** * Test the ConnectionManager with various security settings. diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala new file mode 100644 index 0000000000000..13b415cccb647 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -0,0 +1,417 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import java.lang.ref.WeakReference + +import scala.collection.mutable.{HashSet, SynchronizedSet} +import scala.language.existentials +import scala.language.postfixOps +import scala.util.Random + +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.concurrent.{PatienceConfiguration, Eventually} +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.{BlockId, BroadcastBlockId, RDDBlockId, ShuffleBlockId} + +class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { + + implicit val defaultTimeout = timeout(10000 millis) + val conf = new SparkConf() + .setMaster("local[2]") + .setAppName("ContextCleanerSuite") + .set("spark.cleaner.referenceTracking.blocking", "true") + + before { + sc = new SparkContext(conf) + } + + after { + if (sc != null) { + sc.stop() + sc = null + } + } + + + test("cleanup RDD") { + val rdd = newRDD.persist() + val collected = rdd.collect().toList + val tester = new CleanerTester(sc, rddIds = Seq(rdd.id)) + + // Explicit cleanup + cleaner.doCleanupRDD(rdd.id, blocking = true) + tester.assertCleanup() + + // Verify that RDDs can be re-executed after cleaning up + assert(rdd.collect().toList === collected) + } + + test("cleanup shuffle") { + val (rdd, shuffleDeps) = newRDDWithShuffleDependencies + val collected = rdd.collect().toList + val tester = new CleanerTester(sc, shuffleIds = shuffleDeps.map(_.shuffleId)) + + // Explicit cleanup + shuffleDeps.foreach(s => cleaner.doCleanupShuffle(s.shuffleId, blocking = true)) + tester.assertCleanup() + + // Verify that shuffles can be re-executed after cleaning up + assert(rdd.collect().toList.equals(collected)) + } + + test("cleanup broadcast") { + val broadcast = newBroadcast + val tester = new CleanerTester(sc, broadcastIds = Seq(broadcast.id)) + + // Explicit cleanup + cleaner.doCleanupBroadcast(broadcast.id, blocking = true) + tester.assertCleanup() + } + + test("automatically cleanup RDD") { + var rdd = newRDD.persist() + rdd.count() + + // Test that GC does not cause RDD cleanup due to a strong reference + val preGCTester = new CleanerTester(sc, rddIds = Seq(rdd.id)) + runGC() + intercept[Exception] { + preGCTester.assertCleanup()(timeout(1000 millis)) + } + + // Test that GC causes RDD cleanup after dereferencing the RDD + val postGCTester = new CleanerTester(sc, rddIds = Seq(rdd.id)) + rdd = null // Make RDD out of scope + runGC() + postGCTester.assertCleanup() + } + + test("automatically cleanup shuffle") { + var rdd = newShuffleRDD + rdd.count() + + // Test that GC does not cause shuffle cleanup due to a strong reference + val preGCTester = new CleanerTester(sc, shuffleIds = Seq(0)) + runGC() + intercept[Exception] { + preGCTester.assertCleanup()(timeout(1000 millis)) + } + + // Test that GC causes shuffle cleanup after dereferencing the RDD + val postGCTester = new CleanerTester(sc, shuffleIds = Seq(0)) + rdd = null // Make RDD out of scope, so that corresponding shuffle goes out of scope + runGC() + postGCTester.assertCleanup() + } + + test("automatically cleanup broadcast") { + var broadcast = newBroadcast + + // Test that GC does not cause broadcast cleanup due to a strong reference + val preGCTester = new CleanerTester(sc, broadcastIds = Seq(broadcast.id)) + runGC() + intercept[Exception] { + preGCTester.assertCleanup()(timeout(1000 millis)) + } + + // Test that GC causes broadcast cleanup after dereferencing the broadcast variable + val postGCTester = new CleanerTester(sc, broadcastIds = Seq(broadcast.id)) + broadcast = null // Make broadcast variable out of scope + runGC() + postGCTester.assertCleanup() + } + + test("automatically cleanup RDD + shuffle + broadcast") { + val numRdds = 100 + val numBroadcasts = 4 // Broadcasts are more costly + val rddBuffer = (1 to numRdds).map(i => randomRdd).toBuffer + val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast).toBuffer + val rddIds = sc.persistentRdds.keys.toSeq + val shuffleIds = 0 until sc.newShuffleId + val broadcastIds = 0L until numBroadcasts + + val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) + runGC() + intercept[Exception] { + preGCTester.assertCleanup()(timeout(1000 millis)) + } + + // Test that GC triggers the cleanup of all variables after the dereferencing them + val postGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) + broadcastBuffer.clear() + rddBuffer.clear() + runGC() + postGCTester.assertCleanup() + } + + test("automatically cleanup RDD + shuffle + broadcast in distributed mode") { + sc.stop() + + val conf2 = new SparkConf() + .setMaster("local-cluster[2, 1, 512]") + .setAppName("ContextCleanerSuite") + .set("spark.cleaner.referenceTracking.blocking", "true") + sc = new SparkContext(conf2) + + val numRdds = 10 + val numBroadcasts = 4 // Broadcasts are more costly + val rddBuffer = (1 to numRdds).map(i => randomRdd).toBuffer + val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast).toBuffer + val rddIds = sc.persistentRdds.keys.toSeq + val shuffleIds = 0 until sc.newShuffleId + val broadcastIds = 0L until numBroadcasts + + val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) + runGC() + intercept[Exception] { + preGCTester.assertCleanup()(timeout(1000 millis)) + } + + // Test that GC triggers the cleanup of all variables after the dereferencing them + val postGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) + broadcastBuffer.clear() + rddBuffer.clear() + runGC() + postGCTester.assertCleanup() + } + + //------ Helper functions ------ + + def newRDD = sc.makeRDD(1 to 10) + def newPairRDD = newRDD.map(_ -> 1) + def newShuffleRDD = newPairRDD.reduceByKey(_ + _) + def newBroadcast = sc.broadcast(1 to 100) + def newRDDWithShuffleDependencies: (RDD[_], Seq[ShuffleDependency[_, _, _]]) = { + def getAllDependencies(rdd: RDD[_]): Seq[Dependency[_]] = { + rdd.dependencies ++ rdd.dependencies.flatMap { dep => + getAllDependencies(dep.rdd) + } + } + val rdd = newShuffleRDD + + // Get all the shuffle dependencies + val shuffleDeps = getAllDependencies(rdd) + .filter(_.isInstanceOf[ShuffleDependency[_, _, _]]) + .map(_.asInstanceOf[ShuffleDependency[_, _, _]]) + (rdd, shuffleDeps) + } + + def randomRdd = { + val rdd: RDD[_] = Random.nextInt(3) match { + case 0 => newRDD + case 1 => newShuffleRDD + case 2 => newPairRDD.join(newPairRDD) + } + if (Random.nextBoolean()) rdd.persist() + rdd.count() + rdd + } + + def randomBroadcast = { + sc.broadcast(Random.nextInt(Int.MaxValue)) + } + + /** Run GC and make sure it actually has run */ + def runGC() { + val weakRef = new WeakReference(new Object()) + val startTime = System.currentTimeMillis + System.gc() // Make a best effort to run the garbage collection. It *usually* runs GC. + // Wait until a weak reference object has been GCed + while(System.currentTimeMillis - startTime < 10000 && weakRef.get != null) { + System.gc() + Thread.sleep(200) + } + } + + def cleaner = sc.cleaner.get +} + + +/** Class to test whether RDDs, shuffles, etc. have been successfully cleaned. */ +class CleanerTester( + sc: SparkContext, + rddIds: Seq[Int] = Seq.empty, + shuffleIds: Seq[Int] = Seq.empty, + broadcastIds: 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 isDistributed = !sc.isLocal + + val cleanerListener = new CleanerListener { + def rddCleaned(rddId: Int): Unit = { + toBeCleanedRDDIds -= rddId + logInfo("RDD "+ rddId + " cleaned") + } + + def shuffleCleaned(shuffleId: Int): Unit = { + toBeCleanedShuffleIds -= shuffleId + logInfo("Shuffle " + shuffleId + " cleaned") + } + + def broadcastCleaned(broadcastId: Long): Unit = { + toBeCleanedBroadcstIds -= broadcastId + logInfo("Broadcast" + broadcastId + " cleaned") + } + } + + val MAX_VALIDATION_ATTEMPTS = 10 + val VALIDATION_ATTEMPT_INTERVAL = 100 + + logInfo("Attempting to validate before cleanup:\n" + uncleanedResourcesToString) + preCleanupValidate() + sc.cleaner.get.attachListener(cleanerListener) + + /** Assert that all the stuff has been cleaned up */ + def assertCleanup()(implicit waitTimeout: PatienceConfiguration.Timeout) { + try { + eventually(waitTimeout, interval(100 millis)) { + assert(isAllCleanedUp) + } + postCleanupValidate() + } finally { + logInfo("Resources left from cleaning up:\n" + uncleanedResourcesToString) + } + } + + /** Verify that RDDs, shuffles, etc. occupy resources */ + private def preCleanupValidate() { + assert(rddIds.nonEmpty || shuffleIds.nonEmpty || broadcastIds.nonEmpty, "Nothing to cleanup") + + // Verify the RDDs have been persisted and blocks are present + rddIds.foreach { rddId => + assert( + sc.persistentRdds.contains(rddId), + "RDD " + rddId + " have not been persisted, cannot start cleaner test" + ) + + assert( + !getRDDBlocks(rddId).isEmpty, + "Blocks of RDD " + rddId + " cannot be found in block manager, " + + "cannot start cleaner test" + ) + } + + // Verify the shuffle ids are registered and blocks are present + shuffleIds.foreach { shuffleId => + assert( + mapOutputTrackerMaster.containsShuffle(shuffleId), + "Shuffle " + shuffleId + " have not been registered, cannot start cleaner test" + ) + + assert( + !getShuffleBlocks(shuffleId).isEmpty, + "Blocks of shuffle " + shuffleId + " cannot be found in block manager, " + + "cannot start cleaner test" + ) + } + + // Verify that the broadcast blocks are present + broadcastIds.foreach { broadcastId => + assert( + !getBroadcastBlocks(broadcastId).isEmpty, + "Blocks of broadcast " + broadcastId + "cannot be found in block manager, " + + "cannot start cleaner test" + ) + } + } + + /** + * Verify that RDDs, shuffles, etc. do not occupy resources. Tests multiple times as there is + * as there is not guarantee on how long it will take clean up the resources. + */ + private def postCleanupValidate() { + // Verify the RDDs have been persisted and blocks are present + rddIds.foreach { rddId => + assert( + !sc.persistentRdds.contains(rddId), + "RDD " + rddId + " was not cleared from sc.persistentRdds" + ) + + assert( + getRDDBlocks(rddId).isEmpty, + "Blocks of RDD " + rddId + " were not cleared from block manager" + ) + } + + // Verify the shuffle ids are registered and blocks are present + shuffleIds.foreach { shuffleId => + assert( + !mapOutputTrackerMaster.containsShuffle(shuffleId), + "Shuffle " + shuffleId + " was not deregistered from map output tracker" + ) + + assert( + getShuffleBlocks(shuffleId).isEmpty, + "Blocks of shuffle " + shuffleId + " were not cleared from block manager" + ) + } + + // Verify that the broadcast blocks are present + broadcastIds.foreach { broadcastId => + assert( + getBroadcastBlocks(broadcastId).isEmpty, + "Blocks of broadcast " + broadcastId + " were not cleared from block manager" + ) + } + } + + private def uncleanedResourcesToString = { + s""" + |\tRDDs = ${toBeCleanedRDDIds.toSeq.sorted.mkString("[", ", ", "]")} + |\tShuffles = ${toBeCleanedShuffleIds.toSeq.sorted.mkString("[", ", ", "]")} + |\tBroadcasts = ${toBeCleanedBroadcstIds.toSeq.sorted.mkString("[", ", ", "]")} + """.stripMargin + } + + private def isAllCleanedUp = + toBeCleanedRDDIds.isEmpty && + toBeCleanedShuffleIds.isEmpty && + toBeCleanedBroadcstIds.isEmpty + + private def getRDDBlocks(rddId: Int): Seq[BlockId] = { + blockManager.master.getMatchingBlockIds( _ match { + case RDDBlockId(`rddId`, _) => true + case _ => false + }, askSlaves = true) + } + + private def getShuffleBlocks(shuffleId: Int): Seq[BlockId] = { + blockManager.master.getMatchingBlockIds( _ match { + case ShuffleBlockId(`shuffleId`, _, _) => true + case _ => false + }, askSlaves = true) + } + + private def getBroadcastBlocks(broadcastId: Long): Seq[BlockId] = { + blockManager.master.getMatchingBlockIds( _ match { + case BroadcastBlockId(`broadcastId`, _) => true + case _ => false + }, askSlaves = true) + } + + private def blockManager = sc.env.blockManager + private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] +} diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 14ddd6f1ec08f..41c294f727b3c 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark import org.scalatest.BeforeAndAfter import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts._ -import org.scalatest.matchers.ShouldMatchers +import org.scalatest.Matchers import org.scalatest.time.{Millis, Span} import org.apache.spark.SparkContext._ @@ -31,7 +31,7 @@ class NotSerializableClass class NotSerializableExn(val notSer: NotSerializableClass) extends Throwable() {} -class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter +class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter with LocalSparkContext { val clusterUrl = "local-cluster[2,1,512]" diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 9cbdfc54a3dc8..de4bd90c8f7e5 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -29,6 +29,8 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.util.Utils +import scala.language.postfixOps + class DriverSuite extends FunSuite with Timeouts { test("driver should exit after finishing") { @@ -39,7 +41,7 @@ class DriverSuite extends FunSuite with Timeouts { failAfter(60 seconds) { Utils.executeAndGetOutput( Seq("./bin/spark-class", "org.apache.spark.DriverWithoutCleanup", master), - new File(sparkHome), + new File(sparkHome), Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome)) } } diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index f3fb64d87a2fd..12dbebcb28644 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -72,7 +72,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { throw new Exception("Intentional task failure") } } - (k, v(0) * v(0)) + (k, v.head * v.head) }.collect() FailureSuiteState.synchronized { assert(FailureSuiteState.tasksRun === 4) @@ -137,5 +137,3 @@ class FailureSuite extends FunSuite with LocalSparkContext { // TODO: Need to add tests with shuffle fetch failures. } - - diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index aee9ab9091dac..7e18f45de7b5b 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -24,9 +24,11 @@ import com.google.common.io.Files import org.scalatest.FunSuite import org.apache.spark.SparkContext._ +import org.apache.spark.util.Utils class FileServerSuite extends FunSuite with LocalSparkContext { + @transient var tmpDir: File = _ @transient var tmpFile: File = _ @transient var tmpJarUrl: String = _ @@ -38,22 +40,25 @@ class FileServerSuite extends FunSuite with LocalSparkContext { override def beforeAll() { super.beforeAll() - val tmpDir = new File(Files.createTempDir(), "test") - tmpDir.mkdir() - val textFile = new File(tmpDir, "FileServerSuite.txt") + tmpDir = Files.createTempDir() + tmpDir.deleteOnExit() + val testTempDir = new File(tmpDir, "test") + testTempDir.mkdir() + + val textFile = new File(testTempDir, "FileServerSuite.txt") val pw = new PrintWriter(textFile) pw.println("100") pw.close() - - val jarFile = new File(tmpDir, "test.jar") + + val jarFile = new File(testTempDir, "test.jar") val jarStream = new FileOutputStream(jarFile) val jar = new JarOutputStream(jarStream, new java.util.jar.Manifest()) System.setProperty("spark.authenticate", "false") val jarEntry = new JarEntry(textFile.getName) jar.putNextEntry(jarEntry) - + val in = new FileInputStream(textFile) val buffer = new Array[Byte](10240) var nRead = 0 @@ -70,6 +75,11 @@ class FileServerSuite extends FunSuite with LocalSparkContext { tmpJarUrl = jarFile.toURI.toURL.toString } + override def afterAll() { + super.afterAll() + Utils.deleteRecursively(tmpDir) + } + test("Distributing files locally") { sc = new SparkContext("local[4]", "test") sc.addFile(tmpFile.toString) diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 01af94077144a..070e974657860 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -30,12 +30,24 @@ import org.apache.hadoop.mapreduce.Job import org.scalatest.FunSuite import org.apache.spark.SparkContext._ +import org.apache.spark.util.Utils class FileSuite extends FunSuite with LocalSparkContext { + var tempDir: File = _ + + override def beforeEach() { + super.beforeEach() + tempDir = Files.createTempDir() + tempDir.deleteOnExit() + } + + override def afterEach() { + super.afterEach() + Utils.deleteRecursively(tempDir) + } test("text files") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 4) nums.saveAsTextFile(outputDir) @@ -49,7 +61,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("text files (compressed)") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val normalDir = new File(tempDir, "output_normal").getAbsolutePath val compressedOutputDir = new File(tempDir, "output_compressed").getAbsolutePath val codec = new DefaultCodec() @@ -71,7 +82,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("SequenceFiles") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (x, "a" * x)) // (1,a), (2,aa), (3,aaa) nums.saveAsSequenceFile(outputDir) @@ -82,7 +92,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("SequenceFile (compressed)") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val normalDir = new File(tempDir, "output_normal").getAbsolutePath val compressedOutputDir = new File(tempDir, "output_compressed").getAbsolutePath val codec = new DefaultCodec() @@ -104,9 +113,8 @@ class FileSuite extends FunSuite with LocalSparkContext { test("SequenceFile with writable key") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath - val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), "a" * x)) + val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), "a" * x)) nums.saveAsSequenceFile(outputDir) // Try reading the output back as a SequenceFile val output = sc.sequenceFile[IntWritable, Text](outputDir) @@ -115,7 +123,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("SequenceFile with writable value") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (x, new Text("a" * x))) nums.saveAsSequenceFile(outputDir) @@ -126,7 +133,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("SequenceFile with writable key and value") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), new Text("a" * x))) nums.saveAsSequenceFile(outputDir) @@ -137,7 +143,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("implicit conversions in reading SequenceFiles") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (x, "a" * x)) // (1,a), (2,aa), (3,aaa) nums.saveAsSequenceFile(outputDir) @@ -154,7 +159,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("object files of ints") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 4) nums.saveAsObjectFile(outputDir) @@ -165,7 +169,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("object files of complex types") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (x, "a" * x)) nums.saveAsObjectFile(outputDir) @@ -177,7 +180,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("write SequenceFile using new Hadoop API") { import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), new Text("a" * x))) nums.saveAsNewAPIHadoopFile[SequenceFileOutputFormat[IntWritable, Text]]( @@ -189,7 +191,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("read SequenceFile using new Hadoop API") { import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), new Text("a" * x))) nums.saveAsSequenceFile(outputDir) @@ -200,7 +201,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("file caching") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val out = new FileWriter(tempDir + "/input") out.write("Hello world!\n") out.write("What's up?\n") @@ -214,67 +214,83 @@ class FileSuite extends FunSuite with LocalSparkContext { test ("prevent user from overwriting the empty directory (old Hadoop API)") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 1) intercept[FileAlreadyExistsException] { - randomRDD.saveAsTextFile(tempdir.getPath) + randomRDD.saveAsTextFile(tempDir.getPath) } } test ("prevent user from overwriting the non-empty directory (old Hadoop API)") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 1) - randomRDD.saveAsTextFile(tempdir.getPath + "/output") - assert(new File(tempdir.getPath + "/output/part-00000").exists() === true) + randomRDD.saveAsTextFile(tempDir.getPath + "/output") + assert(new File(tempDir.getPath + "/output/part-00000").exists() === true) intercept[FileAlreadyExistsException] { - randomRDD.saveAsTextFile(tempdir.getPath + "/output") + randomRDD.saveAsTextFile(tempDir.getPath + "/output") } } + test ("allow user to disable the output directory existence checking (old Hadoop API") { + val sf = new SparkConf() + sf.setAppName("test").setMaster("local").set("spark.hadoop.validateOutputSpecs", "false") + sc = new SparkContext(sf) + val randomRDD = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 1) + randomRDD.saveAsTextFile(tempDir.getPath + "/output") + assert(new File(tempDir.getPath + "/output/part-00000").exists() === true) + randomRDD.saveAsTextFile(tempDir.getPath + "/output") + assert(new File(tempDir.getPath + "/output/part-00000").exists() === true) + } + test ("prevent user from overwriting the empty directory (new Hadoop API)") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) intercept[FileAlreadyExistsException] { - randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempdir.getPath) + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath) } } test ("prevent user from overwriting the non-empty directory (new Hadoop API)") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() 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") + assert(new File(tempDir.getPath + "/output/part-r-00000").exists() === true) intercept[FileAlreadyExistsException] { - randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempdir.getPath) + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath) } } + test ("allow user to disable the output directory existence checking (new Hadoop API") { + 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") + assert(new File(tempDir.getPath + "/output/part-r-00000").exists() === true) + 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 tempdir = Files.createTempDir() 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]) job.set("mapred.output.format.class", classOf[TextOutputFormat[String, String]].getName) - job.set("mapred.output.dir", tempdir.getPath + "/outputDataset_old") + job.set("mapred.output.dir", tempDir.getPath + "/outputDataset_old") randomRDD.saveAsHadoopDataset(job) - assert(new File(tempdir.getPath + "/outputDataset_old/part-00000").exists() === true) + assert(new File(tempDir.getPath + "/outputDataset_old/part-00000").exists() === true) } test ("save Hadoop Dataset through new Hadoop API") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() 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]) job.setOutputFormatClass(classOf[NewTextOutputFormat[String, String]]) - job.getConfiguration.set("mapred.output.dir", tempdir.getPath + "/outputDataset_new") + job.getConfiguration.set("mapred.output.dir", tempDir.getPath + "/outputDataset_new") randomRDD.saveAsNewAPIHadoopDataset(job.getConfiguration) - assert(new File(tempdir.getPath + "/outputDataset_new/part-r-00000").exists() === true) + assert(new File(tempDir.getPath + "/outputDataset_new/part-r-00000").exists() === true) } } diff --git a/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala new file mode 100644 index 0000000000000..4bd889135631b --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.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 + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext._ + +class ImplicitOrderingSuite extends FunSuite with LocalSparkContext { + class NonOrderedClass {} + + class ComparableClass extends Comparable[ComparableClass] { + override def compareTo(o: ComparableClass): Int = ??? + } + + class OrderedClass extends Ordered[OrderedClass] { + override def compare(o: OrderedClass): Int = ??? + } + + // Tests that PairRDDFunctions grabs an implicit Ordering in various cases where it should. + test("basic inference of Orderings"){ + sc = new SparkContext("local", "test") + val rdd = sc.parallelize(1 to 10) + + // Infer orderings after basic maps to particular types + assert(rdd.map(x => (x, x)).keyOrdering.isDefined) + assert(rdd.map(x => (1, x)).keyOrdering.isDefined) + assert(rdd.map(x => (x.toString, x)).keyOrdering.isDefined) + assert(rdd.map(x => (null, x)).keyOrdering.isDefined) + assert(rdd.map(x => (new NonOrderedClass, x)).keyOrdering.isEmpty) + assert(rdd.map(x => (new ComparableClass, x)).keyOrdering.isDefined) + assert(rdd.map(x => (new OrderedClass, x)).keyOrdering.isDefined) + + // Infer orderings for other RDD methods + assert(rdd.groupBy(x => x).keyOrdering.isDefined) + assert(rdd.groupBy(x => new NonOrderedClass).keyOrdering.isEmpty) + assert(rdd.groupBy(x => new ComparableClass).keyOrdering.isDefined) + assert(rdd.groupBy(x => new OrderedClass).keyOrdering.isDefined) + assert(rdd.groupBy((x: Int) => x, 5).keyOrdering.isDefined) + assert(rdd.groupBy((x: Int) => x, new HashPartitioner(5)).keyOrdering.isDefined) + } +} diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 7a39d1af9e2d5..a57430e829ced 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -21,11 +21,11 @@ import java.util.concurrent.Semaphore import scala.concurrent.Await import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent.duration.Duration +import scala.concurrent.duration._ import scala.concurrent.future import org.scalatest.{BeforeAndAfter, FunSuite} -import org.scalatest.matchers.ShouldMatchers +import org.scalatest.Matchers import org.apache.spark.SparkContext._ import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart} @@ -35,7 +35,7 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart} * (e.g. count) as well as multi-job action (e.g. take). We test the local and cluster schedulers * in both FIFO and fair scheduling modes. */ -class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAfter +class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter with LocalSparkContext { override def afterEach() { @@ -84,6 +84,35 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf assert(sc.parallelize(1 to 10, 2).count === 10) } + test("do not put partially executed partitions into cache") { + // In this test case, we create a scenario in which a partition is only partially executed, + // and make sure CacheManager does not put that partially executed partition into the + // BlockManager. + import JobCancellationSuite._ + sc = new SparkContext("local", "test") + + // Run from 1 to 10, and then block and wait for the task to be killed. + val rdd = sc.parallelize(1 to 1000, 2).map { x => + if (x > 10) { + taskStartedSemaphore.release() + taskCancelledSemaphore.acquire() + } + x + }.cache() + + val rdd1 = rdd.map(x => x) + + future { + taskStartedSemaphore.acquire() + sc.cancelAllJobs() + taskCancelledSemaphore.release(100000) + } + + intercept[SparkException] { rdd1.count() } + // If the partial block is put into cache, rdd.count() would return a number less than 1000. + assert(rdd.count() === 1000) + } + test("job group") { sc = new SparkContext("local[2]", "test") @@ -101,27 +130,57 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.count() } + // Block until both tasks of job A have started and cancel job A. + sem.acquire(2) + sc.clearJobGroup() val jobB = sc.parallelize(1 to 100, 2).countAsync() + sc.cancelJobGroup("jobA") + val e = intercept[SparkException] { Await.result(jobA, Duration.Inf) } + assert(e.getMessage contains "cancel") + + // Once A is cancelled, job B should finish fairly quickly. + assert(jobB.get() === 100) + } + + test("job group with interruption") { + 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() + } + }) + + // jobA is the one to be cancelled. + val jobA = future { + sc.setJobGroup("jobA", "this is a job to be cancelled", interruptOnCancel = true) + sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(100000); i }.count() + } // Block until both tasks of job A have started and cancel job A. sem.acquire(2) + + sc.clearJobGroup() + val jobB = sc.parallelize(1 to 100, 2).countAsync() sc.cancelJobGroup("jobA") - val e = intercept[SparkException] { Await.result(jobA, Duration.Inf) } + val e = intercept[SparkException] { Await.result(jobA, 5.seconds) } assert(e.getMessage contains "cancel") // Once A is cancelled, job B should finish fairly quickly. assert(jobB.get() === 100) } -/* - test("two jobs sharing the same stage") { + + ignore("two jobs sharing the same stage") { // sem1: make sure cancel is issued after some tasks are launched // sem2: make sure the first stage is not finished until cancel is issued val sem1 = new Semaphore(0) val sem2 = new Semaphore(0) sc = new SparkContext("local[2]", "test") - sc.dagScheduler.addSparkListener(new SparkListener { + sc.addSparkListener(new SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { sem1.release() } @@ -147,7 +206,7 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf intercept[SparkException] { f1.get() } intercept[SparkException] { f2.get() } } - */ + def testCount() { // Cancel before launching any tasks { @@ -206,3 +265,9 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf } } } + + +object JobCancellationSuite { + val taskStartedSemaphore = new Semaphore(0) + val taskCancelledSemaphore = new Semaphore(0) +} diff --git a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala index 4b972f88a9542..53e367a61715b 100644 --- a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala @@ -17,8 +17,7 @@ package org.apache.spark -import org.jboss.netty.logging.InternalLoggerFactory -import org.jboss.netty.logging.Slf4JLoggerFactory +import _root_.io.netty.util.internal.logging.{Slf4JLoggerFactory, InternalLoggerFactory} import org.scalatest.BeforeAndAfterAll import org.scalatest.BeforeAndAfterEach import org.scalatest.Suite diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index a5bd72eb0a122..95ba273f16a71 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -57,12 +57,13 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { tracker.stop() } - test("master register and fetch") { + test("master register shuffle and fetch") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTrackerMaster(conf) tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) tracker.registerShuffle(10, 2) + assert(tracker.containsShuffle(10)) val compressedSize1000 = MapOutputTracker.compressSize(1000L) val compressedSize10000 = MapOutputTracker.compressSize(10000L) val size1000 = MapOutputTracker.decompressSize(compressedSize1000) @@ -77,7 +78,25 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { tracker.stop() } - test("master register and unregister and fetch") { + test("master register and unregister shuffle") { + val actorSystem = ActorSystem("test") + val tracker = new MapOutputTrackerMaster(conf) + tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) + tracker.registerShuffle(10, 2) + val compressedSize1000 = MapOutputTracker.compressSize(1000L) + val compressedSize10000 = MapOutputTracker.compressSize(10000L) + tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000, 0), + Array(compressedSize1000, compressedSize10000))) + tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000, 0), + Array(compressedSize10000, compressedSize1000))) + assert(tracker.containsShuffle(10)) + assert(tracker.getServerStatuses(10, 0).nonEmpty) + tracker.unregisterShuffle(10) + assert(!tracker.containsShuffle(10)) + assert(tracker.getServerStatuses(10, 0).isEmpty) + } + + test("master register shuffle and unregister map output and fetch") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTrackerMaster(conf) tracker.trackerActor = @@ -105,16 +124,13 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, securityManager = new SecurityManager(conf)) - // Will be cleared by LocalSparkContext - System.setProperty("spark.driver.port", boundPort.toString) - val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf, securityManager = new SecurityManager(conf)) - val slaveTracker = new MapOutputTracker(conf) + val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala index 627e9b5cd9060..db56a4acdd6f5 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark import java.io.File -import com.google.common.io.Files - import org.scalatest.FunSuite import org.apache.spark.rdd.{HadoopRDD, PipedRDD, HadoopPartition} @@ -28,6 +26,7 @@ import org.apache.hadoop.mapred.{JobConf, TextInputFormat, FileSplit} import org.apache.hadoop.fs.Path import scala.collection.Map +import scala.language.postfixOps import scala.sys.process._ import scala.util.Try @@ -85,7 +84,7 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { (f: String => Unit) => { bl.value.map(f(_)); f("\u0001") }, - (i: Tuple2[String, Seq[String]], f: String => Unit) => { + (i: Tuple2[String, Iterable[String]], f: String => Unit) => { for (e <- i._2) { f(e + "_") } @@ -179,14 +178,12 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { } val hadoopPart1 = generateFakeHadoopPartition() val pipedRdd = new PipedRDD(nums, "printenv " + varName) - val tContext = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = TaskMetrics.empty()) + val tContext = new TaskContext(0, 0, 0) val rddIter = pipedRdd.compute(hadoopPart1, tContext) val arr = rddIter.toArray assert(arr(0) == "/some/path") } else { // printenv isn't available so just pass the test - assert(true) } } diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala new file mode 100644 index 0000000000000..e39093e24d68a --- /dev/null +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.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 + +import scala.collection.mutable.ArrayBuffer + +import org.scalatest.FunSuite + +class SecurityManagerSuite extends FunSuite { + + test("set security with conf") { + val conf = new SparkConf + conf.set("spark.authenticate", "true") + conf.set("spark.authenticate.secret", "good") + conf.set("spark.ui.acls.enable", "true") + conf.set("spark.ui.view.acls", "user1,user2") + val securityManager = new SecurityManager(conf); + assert(securityManager.isAuthenticationEnabled() === true) + assert(securityManager.uiAclsEnabled() === true) + assert(securityManager.checkUIViewPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user2") === true) + assert(securityManager.checkUIViewPermissions("user3") === false) + } + + test("set security with api") { + val conf = new SparkConf + conf.set("spark.ui.view.acls", "user1,user2") + val securityManager = new SecurityManager(conf); + securityManager.setUIAcls(true) + assert(securityManager.uiAclsEnabled() === true) + securityManager.setUIAcls(false) + assert(securityManager.uiAclsEnabled() === false) + + // acls are off so doesn't matter what view acls set to + assert(securityManager.checkUIViewPermissions("user4") === true) + + securityManager.setUIAcls(true) + assert(securityManager.uiAclsEnabled() === true) + securityManager.setViewAcls(ArrayBuffer[String]("user5"), "user6,user7") + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user5") === true) + assert(securityManager.checkUIViewPermissions("user6") === true) + assert(securityManager.checkUIViewPermissions("user7") === true) + assert(securityManager.checkUIViewPermissions("user8") === false) + assert(securityManager.checkUIViewPermissions(null) === true) + } +} + diff --git a/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala index 29d428aa7dc41..47df00050c1e2 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala @@ -23,11 +23,11 @@ class ShuffleNettySuite extends ShuffleSuite with BeforeAndAfterAll { // This test suite should run all tests in ShuffleSuite with Netty shuffle mode. - override def beforeAll(configMap: Map[String, Any]) { + override def beforeAll() { System.setProperty("spark.shuffle.use.netty", "true") } - override def afterAll(configMap: Map[String, Any]) { + override def afterAll() { System.setProperty("spark.shuffle.use.netty", "false") } } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index be6508a40ea61..47112ce66d695 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers +import org.scalatest.Matchers import org.apache.spark.SparkContext._ import org.apache.spark.ShuffleSuite.NonJavaSerializableClass @@ -26,7 +26,7 @@ import org.apache.spark.rdd.{CoGroupedRDD, OrderedRDDFunctions, RDD, ShuffledRDD import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.MutablePair -class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { +class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { val conf = new SparkConf(loadDefaults = false) @@ -58,7 +58,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { // default Java serializer cannot handle the non serializable class. val c = new ShuffledRDD[Int, NonJavaSerializableClass, (Int, NonJavaSerializableClass)]( b, new HashPartitioner(NUM_BLOCKS)).setSerializer(new KryoSerializer(conf)) - val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId + val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId assert(c.count === 10) @@ -97,7 +97,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { val c = new ShuffledRDD[Int, Int, (Int, Int)](b, new HashPartitioner(10)) .setSerializer(new KryoSerializer(conf)) - val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId + val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId assert(c.count === 4) val blockSizes = (0 until NUM_BLOCKS).flatMap { id => @@ -122,7 +122,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { // NOTE: The default Java serializer should create zero-sized blocks val c = new ShuffledRDD[Int, Int, (Int, Int)](b, new HashPartitioner(10)) - val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId + val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId assert(c.count === 4) val blockSizes = (0 until NUM_BLOCKS).flatMap { id => diff --git a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala index cd3887dcc7371..1fde4badda949 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala @@ -70,7 +70,7 @@ package object testPackage extends Assertions { def runCallSiteTest(sc: SparkContext) { val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2) val rddCreationSite = rdd.getCreationSite - val curCallSite = sc.getCallSite() // note: 2 lines after definition of "rdd" + val curCallSite = sc.getCallSite().short // note: 2 lines after definition of "rdd" val rddCreationLine = rddCreationSite match { case CALL_SITE_REGEX(func, file, line) => { diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index b543471a5d35b..94fba102865b3 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -51,6 +51,14 @@ class SparkContextSchedulerCreationSuite } } + test("local-*") { + val sched = createTaskScheduler("local[*]") + sched.backend match { + case s: LocalBackend => assert(s.totalCores === Runtime.getRuntime.availableProcessors()) + case _ => fail() + } + } + test("local-n") { val sched = createTaskScheduler("local[5]") assert(sched.maxTaskFailures === 1) 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 d6b93f5fedd3b..4161aede1d1d0 100644 --- a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala @@ -18,9 +18,9 @@ package org.apache.spark.deploy import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers +import org.scalatest.Matchers -class ClientSuite extends FunSuite with ShouldMatchers { +class ClientSuite extends FunSuite with Matchers { test("correctly validates driver jar URL's") { ClientArguments.isValidJarUrl("http://someHost:8080/foo.jar") should be (true) ClientArguments.isValidJarUrl("file://some/path/to/a/jarFile.jar") should be (true) 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 9f2924c23b73c..01ab2d549325c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -28,6 +28,7 @@ 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 class JsonProtocolSuite extends FunSuite { @@ -87,7 +88,7 @@ class JsonProtocolSuite extends FunSuite { } def createAppDesc(): ApplicationDescription = { - val cmd = new Command("mainClass", List("arg1", "arg2"), Map()) + val cmd = new Command("mainClass", List("arg1", "arg2"), Map(), Seq(), Seq()) new ApplicationDescription("name", Some(4), 1234, cmd, Some("sparkHome"), "appUiUrl") } @@ -100,7 +101,7 @@ class JsonProtocolSuite extends FunSuite { def createDriverCommand() = new Command( "org.apache.spark.FakeClass", Seq("some arg --and-some options -g foo"), - Map(("K1", "V1"), ("K2", "V2")) + Map(("K1", "V1"), ("K2", "V2")), Seq("cp1", "cp2"), Seq("lp1", "lp2"), Some("-Dfoo") ) def createDriverDesc() = new DriverDescription("hdfs://some-dir/some.jar", 100, 3, @@ -116,7 +117,8 @@ class JsonProtocolSuite extends FunSuite { } def createExecutorRunner(): ExecutorRunner = { new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host", - new File("sparkHome"), new File("workDir"), "akka://worker", ExecutorState.RUNNING) + new File("sparkHome"), new File("workDir"), "akka://worker", + new SparkConf, ExecutorState.RUNNING) } def createDriverRunner(): DriverRunner = { new DriverRunner("driverId", new File("workDir"), new File("sparkHome"), createDriverDesc(), @@ -133,9 +135,12 @@ class JsonProtocolSuite extends FunSuite { def assertValidDataInJson(validateJson: JValue, expectedJson: JValue) { val Diff(c, a, d) = validateJson diff expectedJson - assert(c === JNothing, "Json changed") - assert(a === JNothing, "Json added") - assert(d === JNothing, "Json deleted") + 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)}") } } @@ -165,7 +170,7 @@ object JsonConstants { """ |{"name":"name","cores":4,"memoryperslave":1234, |"user":"%s","sparkhome":"sparkHome", - |"command":"Command(mainClass,List(arg1, arg2),Map())"} + |"command":"Command(mainClass,List(arg1, arg2),Map(),List(),List(),None)"} """.format(System.getProperty("user.name", "")).stripMargin val executorRunnerJsonStr = diff --git a/core/src/test/scala/org/apache/spark/deploy/PythonRunnerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/PythonRunnerSuite.scala new file mode 100644 index 0000000000000..bb6251fb4bfbe --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/PythonRunnerSuite.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.deploy + +import org.scalatest.FunSuite + +class PythonRunnerSuite extends FunSuite { + + // Test formatting a single path to be added to the PYTHONPATH + test("format path") { + assert(PythonRunner.formatPath("spark.py") === "spark.py") + assert(PythonRunner.formatPath("file:/spark.py") === "/spark.py") + assert(PythonRunner.formatPath("file:///spark.py") === "/spark.py") + assert(PythonRunner.formatPath("local:/spark.py") === "/spark.py") + assert(PythonRunner.formatPath("local:///spark.py") === "/spark.py") + assert(PythonRunner.formatPath("C:/a/b/spark.py", testWindows = true) === "C:/a/b/spark.py") + assert(PythonRunner.formatPath("/C:/a/b/spark.py", testWindows = true) === "C:/a/b/spark.py") + assert(PythonRunner.formatPath("file:/C:/a/b/spark.py", testWindows = true) === + "C:/a/b/spark.py") + intercept[IllegalArgumentException] { PythonRunner.formatPath("one:two") } + intercept[IllegalArgumentException] { PythonRunner.formatPath("hdfs:s3:xtremeFS") } + intercept[IllegalArgumentException] { PythonRunner.formatPath("hdfs:/path/to/some.py") } + } + + // Test formatting multiple comma-separated paths to be added to the PYTHONPATH + test("format paths") { + assert(PythonRunner.formatPaths("spark.py") === Array("spark.py")) + assert(PythonRunner.formatPaths("file:/spark.py") === Array("/spark.py")) + assert(PythonRunner.formatPaths("file:/app.py,local:/spark.py") === + Array("/app.py", "/spark.py")) + assert(PythonRunner.formatPaths("me.py,file:/you.py,local:/we.py") === + Array("me.py", "/you.py", "/we.py")) + assert(PythonRunner.formatPaths("C:/a/b/spark.py", testWindows = true) === + Array("C:/a/b/spark.py")) + assert(PythonRunner.formatPaths("/C:/a/b/spark.py", testWindows = true) === + Array("C:/a/b/spark.py")) + assert(PythonRunner.formatPaths("C:/free.py,pie.py", testWindows = true) === + Array("C:/free.py", "pie.py")) + assert(PythonRunner.formatPaths("lovely.py,C:/free.py,file:/d:/fry.py", testWindows = true) === + Array("lovely.py", "C:/free.py", "d:/fry.py")) + intercept[IllegalArgumentException] { PythonRunner.formatPaths("one:two,three") } + intercept[IllegalArgumentException] { PythonRunner.formatPaths("two,three,four:five:six") } + intercept[IllegalArgumentException] { PythonRunner.formatPaths("hdfs:/some.py,foo.py") } + intercept[IllegalArgumentException] { PythonRunner.formatPaths("foo.py,hdfs:/some.py") } + } +} 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 4e489cd9b66a6..565c53e9529ff 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -17,17 +17,20 @@ package org.apache.spark.deploy -import java.io.{OutputStream, PrintStream} +import java.io.{File, OutputStream, PrintStream} import scala.collection.mutable.ArrayBuffer -import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers - +import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkException, TestUtils} import org.apache.spark.deploy.SparkSubmit._ +import org.apache.spark.util.Utils +import org.scalatest.FunSuite +import org.scalatest.Matchers - -class SparkSubmitSuite extends FunSuite with ShouldMatchers { +class SparkSubmitSuite extends FunSuite with Matchers { + def beforeAll() { + System.setProperty("spark.testing", "true") + } val noOpOutputStream = new OutputStream { def write(b: Int) = {} @@ -42,7 +45,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } /** Returns true if the script exits and the given search string is printed. */ - def testPrematureExit(input: Array[String], searchString: String): Boolean = { + def testPrematureExit(input: Array[String], searchString: String) = { val printStream = new BufferPrintStream() SparkSubmit.printStream = printStream @@ -60,119 +63,258 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } thread.start() thread.join() - printStream.lineBuffer.find(s => s.contains(searchString)).size > 0 + val joined = printStream.lineBuffer.mkString("\n") + if (!joined.contains(searchString)) { + fail(s"Search string '$searchString' not found in $joined") + } } test("prints usage on empty input") { - testPrematureExit(Array[String](), "Usage: spark-submit") should be (true) + testPrematureExit(Array[String](), "Usage: spark-submit") } test("prints usage with only --help") { - testPrematureExit(Array("--help"), "Usage: spark-submit") should be (true) + testPrematureExit(Array("--help"), "Usage: spark-submit") } - test("handles multiple binary definitions") { - val adjacentJars = Array("foo.jar", "bar.jar") - testPrematureExit(adjacentJars, "error: Found two conflicting resources") should be (true) - - val nonAdjacentJars = - Array("foo.jar", "--master", "123", "--class", "abc", "bar.jar") - testPrematureExit(nonAdjacentJars, "error: Found two conflicting resources") should be (true) + test("prints error with unrecognized options") { + testPrematureExit(Array("--blarg"), "Unrecognized option '--blarg'") + testPrematureExit(Array("-bleg"), "Unrecognized option '-bleg'") } test("handle binary specified but not class") { - testPrematureExit(Array("foo.jar"), "must specify a main class") + testPrematureExit(Array("foo.jar"), "No main class") + } + + test("handles arguments with --key=val") { + val clArgs = Seq( + "--jars=one.jar,two.jar,three.jar", + "--name=myApp") + val appArgs = new SparkSubmitArguments(clArgs) + appArgs.jars should include regex (".*one.jar,.*two.jar,.*three.jar") + appArgs.name should be ("myApp") + } + + test("handles arguments to user program") { + val clArgs = Seq( + "--name", "myApp", + "--class", "Foo", + "userjar.jar", + "some", + "--weird", "args") + val appArgs = new SparkSubmitArguments(clArgs) + appArgs.childArgs should be (Seq("some", "--weird", "args")) } test("handles YARN cluster mode") { - val clArgs = Array("thejar.jar", "--deploy-mode", "cluster", - "--master", "yarn", "--executor-memory", "5g", "--executor-cores", "5", - "--class", "org.SomeClass", "--jars", "one.jar,two.jar,three.jar", - "--arg", "arg1", "--arg", "arg2", "--driver-memory", "4g", - "--queue", "thequeue", "--files", "file1.txt,file2.txt", - "--archives", "archive1.txt,archive2.txt", "--num-executors", "6") + val clArgs = Seq( + "--deploy-mode", "cluster", + "--master", "yarn", + "--executor-memory", "5g", + "--executor-cores", "5", + "--class", "org.SomeClass", + "--jars", "one.jar,two.jar,three.jar", + "--driver-memory", "4g", + "--queue", "thequeue", + "--files", "file1.txt,file2.txt", + "--archives", "archive1.txt,archive2.txt", + "--num-executors", "6", + "--name", "beauty", + "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) val childArgsStr = childArgs.mkString(" ") - childArgsStr should include ("--jar thejar.jar") childArgsStr should include ("--class org.SomeClass") - childArgsStr should include ("--addJars one.jar,two.jar,three.jar") childArgsStr should include ("--executor-memory 5g") childArgsStr should include ("--driver-memory 4g") childArgsStr should include ("--executor-cores 5") - childArgsStr should include ("--args arg1 --args arg2") + childArgsStr should include ("--arg arg1 --arg arg2") childArgsStr should include ("--queue thequeue") - childArgsStr should include ("--files file1.txt,file2.txt") - childArgsStr should include ("--archives archive1.txt,archive2.txt") childArgsStr should include ("--num-executors 6") + childArgsStr should include regex ("--jar .*thejar.jar") + childArgsStr should include regex ("--addJars .*one.jar,.*two.jar,.*three.jar") + childArgsStr should include regex ("--files .*file1.txt,.*file2.txt") + childArgsStr should include regex ("--archives .*archive1.txt,.*archive2.txt") mainClass should be ("org.apache.spark.deploy.yarn.Client") classpath should have length (0) - sysProps should have size (0) + sysProps("spark.app.name") should be ("beauty") + sysProps("SPARK_SUBMIT") should be ("true") } test("handles YARN client mode") { - val clArgs = Array("thejar.jar", "--deploy-mode", "client", - "--master", "yarn", "--executor-memory", "5g", "--executor-cores", "5", - "--class", "org.SomeClass", "--jars", "one.jar,two.jar,three.jar", - "--arg", "arg1", "--arg", "arg2", "--driver-memory", "4g", - "--queue", "thequeue", "--files", "file1.txt,file2.txt", - "--archives", "archive1.txt,archive2.txt", "--num-executors", "6") + val clArgs = Seq( + "--deploy-mode", "client", + "--master", "yarn", + "--executor-memory", "5g", + "--executor-cores", "5", + "--class", "org.SomeClass", + "--jars", "one.jar,two.jar,three.jar", + "--driver-memory", "4g", + "--queue", "thequeue", + "--files", "file1.txt,file2.txt", + "--archives", "archive1.txt,archive2.txt", + "--num-executors", "6", + "--name", "trill", + "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") mainClass should be ("org.SomeClass") - classpath should contain ("thejar.jar") - classpath should contain ("one.jar") - classpath should contain ("two.jar") - classpath should contain ("three.jar") + classpath should have length (4) + classpath(0) should endWith ("thejar.jar") + classpath(1) should endWith ("one.jar") + classpath(2) should endWith ("two.jar") + classpath(3) should endWith ("three.jar") + sysProps("spark.app.name") should be ("trill") sysProps("spark.executor.memory") should be ("5g") sysProps("spark.executor.cores") should be ("5") sysProps("spark.yarn.queue") should be ("thequeue") - sysProps("spark.yarn.dist.files") should be ("file1.txt,file2.txt") - sysProps("spark.yarn.dist.archives") should be ("archive1.txt,archive2.txt") sysProps("spark.executor.instances") should be ("6") + sysProps("spark.yarn.dist.files") should include regex (".*file1.txt,.*file2.txt") + sysProps("spark.yarn.dist.archives") should include regex (".*archive1.txt,.*archive2.txt") + sysProps("spark.jars") should include regex (".*one.jar,.*two.jar,.*three.jar,.*thejar.jar") + sysProps("SPARK_SUBMIT") should be ("true") } test("handles standalone cluster mode") { - val clArgs = Array("thejar.jar", "--deploy-mode", "cluster", - "--master", "spark://h:p", "--class", "org.SomeClass", "--arg", "arg1", "--arg", "arg2", - "--supervise", "--driver-memory", "4g", "--driver-cores", "5") + val clArgs = Seq( + "--deploy-mode", "cluster", + "--master", "spark://h:p", + "--class", "org.SomeClass", + "--supervise", + "--driver-memory", "4g", + "--driver-cores", "5", + "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) val childArgsStr = childArgs.mkString(" ") - print("child args: " + childArgsStr) - childArgsStr.startsWith("--memory 4g --cores 5 --supervise") should be (true) - childArgsStr should include ("launch spark://h:p thejar.jar org.SomeClass arg1 arg2") + childArgsStr should startWith ("--memory 4g --cores 5 --supervise") + childArgsStr should include regex ("launch spark://h:p .*thejar.jar org.SomeClass arg1 arg2") mainClass should be ("org.apache.spark.deploy.Client") - classpath should have length (0) - sysProps should have size (0) + classpath should have size (0) + sysProps should have size (2) + sysProps.keys should contain ("spark.jars") + sysProps.keys should contain ("SPARK_SUBMIT") } test("handles standalone client mode") { - val clArgs = Array("thejar.jar", "--deploy-mode", "client", - "--master", "spark://h:p", "--executor-memory", "5g", "--total-executor-cores", "5", - "--class", "org.SomeClass", "--arg", "arg1", "--arg", "arg2", - "--driver-memory", "4g") + val clArgs = Seq( + "--deploy-mode", "client", + "--master", "spark://h:p", + "--executor-memory", "5g", + "--total-executor-cores", "5", + "--class", "org.SomeClass", + "--driver-memory", "4g", + "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") mainClass should be ("org.SomeClass") - classpath should contain ("thejar.jar") + classpath should have length (1) + classpath(0) should endWith ("thejar.jar") sysProps("spark.executor.memory") should be ("5g") sysProps("spark.cores.max") should be ("5") } test("handles mesos client mode") { - val clArgs = Array("thejar.jar", "--deploy-mode", "client", - "--master", "mesos://h:p", "--executor-memory", "5g", "--total-executor-cores", "5", - "--class", "org.SomeClass", "--arg", "arg1", "--arg", "arg2", - "--driver-memory", "4g") + val clArgs = Seq( + "--deploy-mode", "client", + "--master", "mesos://h:p", + "--executor-memory", "5g", + "--total-executor-cores", "5", + "--class", "org.SomeClass", + "--driver-memory", "4g", + "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") mainClass should be ("org.SomeClass") - classpath should contain ("thejar.jar") + classpath should have length (1) + classpath(0) should endWith ("thejar.jar") sysProps("spark.executor.memory") should be ("5g") sysProps("spark.cores.max") should be ("5") } + + test("launch simple application with spark-submit") { + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val args = Seq( + "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local", + unusedJar.toString) + runSparkSubmit(args) + } + + test("spark submit includes jars passed in through --jar") { + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val jar1 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassA")) + val jar2 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassB")) + val jarsString = Seq(jar1, jar2).map(j => j.toString).mkString(",") + val args = Seq( + "--class", JarCreationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local-cluster[2,1,512]", + "--jars", jarsString, + unusedJar.toString) + runSparkSubmit(args) + } + + // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. + def runSparkSubmit(args: Seq[String]): String = { + val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get + Utils.executeAndGetOutput( + Seq("./bin/spark-submit") ++ args, + new File(sparkHome), + Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome)) + } +} + +object JarCreationTest { + def main(args: Array[String]) { + val conf = new SparkConf() + val sc = new SparkContext(conf) + val result = sc.makeRDD(1 to 100, 10).mapPartitions { x => + var foundClasses = false + try { + Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader) + Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader) + foundClasses = true + } catch { + case _: Throwable => // catch all + } + Seq(foundClasses).iterator + }.collect() + if (result.contains(false)) { + throw new Exception("Could not load user defined classes inside of executors") + } + } +} + +object SimpleApplicationTest { + def main(args: Array[String]) { + val conf = new SparkConf() + val sc = new SparkContext(conf) + val configs = Seq("spark.master", "spark.app.name") + for (config <- configs) { + val masterValue = conf.get(config) + val executorValues = sc + .makeRDD(1 to 100, 10) + .map(x => SparkEnv.get.conf.get(config)) + .collect() + .distinct + if (executorValues.size != 1) { + throw new SparkException(s"Inconsistent values for $config: $executorValues") + } + val executorValue = executorValues(0) + if (executorValue != masterValue) { + throw new SparkException( + s"Master had $config=$masterValue but executor had $config=$executorValue") + } + } + } } 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 a2c131b0c9787..4633bc3f7f25e 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 @@ -29,7 +29,7 @@ import org.apache.spark.deploy.{Command, DriverDescription} class DriverRunnerTest extends FunSuite { private def createDriverRunner() = { - val command = new Command("mainClass", Seq(), Map()) + val command = new Command("mainClass", Seq(), Map(), Seq(), Seq()) val driverDescription = new DriverDescription("jarUrl", 512, 1, true, command) new DriverRunner("driverId", new File("workDir"), new File("sparkHome"), driverDescription, null, "akka://1.2.3.4/worker/") 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 3cab8e7b37934..e5f748d55500d 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 @@ -22,16 +22,18 @@ import java.io.File import org.scalatest.FunSuite import org.apache.spark.deploy.{ApplicationDescription, Command, ExecutorState} +import org.apache.spark.SparkConf class ExecutorRunnerTest extends FunSuite { test("command includes appId") { def f(s:String) = new File(s) val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")) - val appDesc = new ApplicationDescription("app name", Some(8), 500, Command("foo", Seq(),Map()), + val appDesc = new ApplicationDescription("app name", Some(8), 500, + Command("foo", Seq(), Map(), Seq(), Seq()), sparkHome, "appUiUrl") val appId = "12345-worker321-9876" val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", f(sparkHome.getOrElse(".")), - f("ooga"), "blah", ExecutorState.RUNNING) + f("ooga"), "blah", new SparkConf, ExecutorState.RUNNING) assert(er.getCommandSeq.last === appId) } 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 0b5ed6d77034b..5e538d6fab2a1 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 @@ -45,4 +45,4 @@ class WorkerWatcherSuite extends FunSuite { actorRef.underlyingActor.receive(new DisassociatedEvent(null, otherAkkaAddress, false)) assert(!actorRef.underlyingActor.isShutDown) } -} \ No newline at end of file +} diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala new file mode 100644 index 0000000000000..e2050e95a1b88 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.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.executor + +import java.net.URLClassLoader + +import org.scalatest.FunSuite + +import org.apache.spark.{LocalSparkContext, SparkContext, SparkException, TestUtils} +import org.apache.spark.util.Utils + +class ExecutorURLClassLoaderSuite extends FunSuite { + + val childClassNames = List("FakeClass1", "FakeClass2") + val parentClassNames = List("FakeClass1", "FakeClass2", "FakeClass3") + val urls = List(TestUtils.createJarWithClasses(childClassNames, "1")).toArray + val urls2 = List(TestUtils.createJarWithClasses(parentClassNames, "2")).toArray + + test("child first") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader) + val fakeClass = classLoader.loadClass("FakeClass2").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "1") + } + + test("parent first") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ExecutorURLClassLoader(urls, parentLoader) + val fakeClass = classLoader.loadClass("FakeClass1").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "2") + } + + test("child first can fall back") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader) + val fakeClass = classLoader.loadClass("FakeClass3").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "2") + } + + test("child first can fail") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader) + intercept[java.lang.ClassNotFoundException] { + classLoader.loadClass("FakeClassDoesNotExist").newInstance() + } + } + + test("driver sets context class loader in local mode") { + // Test the case where the driver program sets a context classloader and then runs a job + // in local mode. This is what happens when ./spark-submit is called with "local" as the + // master. + val original = Thread.currentThread().getContextClassLoader + + val className = "ClassForDriverTest" + val jar = TestUtils.createJarWithClasses(Seq(className)) + val contextLoader = new URLClassLoader(Array(jar), Utils.getContextOrSparkClassLoader) + Thread.currentThread().setContextClassLoader(contextLoader) + + val sc = new SparkContext("local", "driverLoaderTest") + + try { + sc.makeRDD(1 to 5, 2).mapPartitions { x => + val loader = Thread.currentThread().getContextClassLoader + Class.forName(className, true, loader).newInstance() + Seq().iterator + }.count() + } + catch { + case e: SparkException if e.getMessage.contains("ClassNotFoundException") => + fail("Local executor could not find class", e) + case t: Throwable => fail("Unexpected exception ", t) + } + + sc.stop() + Thread.currentThread().setContextClassLoader(original) + } +} diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala index 09e35bfc8f85f..d5ebfb3f3fae1 100644 --- a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala @@ -31,6 +31,7 @@ import org.scalatest.FunSuite import org.apache.hadoop.io.Text import org.apache.spark.SparkContext +import org.apache.spark.util.Utils /** * Tests the correctness of @@ -42,7 +43,7 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { override def beforeAll() { sc = new SparkContext("local", "test") - + // Set the block size of local file system to test whether files are split right or not. sc.hadoopConfiguration.setLong("fs.local.block.size", 32) } @@ -67,13 +68,14 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { test("Correctness of WholeTextFileRecordReader.") { val dir = Files.createTempDir() + dir.deleteOnExit() println(s"Local disk address is ${dir.toString}.") WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) => createNativeFile(dir, filename, contents) } - val res = sc.wholeTextFiles(dir.toString).collect() + val res = sc.wholeTextFiles(dir.toString, 3).collect() assert(res.size === WholeTextFileRecordReaderSuite.fileNames.size, "Number of files read out does not fit with the actual value.") @@ -86,7 +88,7 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { s"file $filename contents can not match.") } - dir.delete() + Utils.deleteRecursively(dir) } } diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala index 3b833f2e41867..28197657e9bad 100644 --- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala @@ -22,6 +22,7 @@ import java.util.concurrent.Semaphore import scala.concurrent.{Await, TimeoutException} import scala.concurrent.duration.Duration import scala.concurrent.ExecutionContext.Implicits.global +import scala.language.postfixOps import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.scalatest.concurrent.Timeouts 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 7c7f69b261a0a..76e317d754ba3 100644 --- a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.rdd import java.sql._ import org.scalatest.{BeforeAndAfter, FunSuite} -import org.apache.spark.rdd.JdbcRDD +import org.apache.spark.{LocalSparkContext, SparkContext} class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { @@ -35,18 +35,18 @@ class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { ID INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1), DATA INTEGER )""") - create.close + create.close() val insert = conn.prepareStatement("INSERT INTO FOO(DATA) VALUES(?)") (1 to 100).foreach { i => insert.setInt(1, i * 2) insert.executeUpdate } - insert.close + insert.close() } catch { case e: SQLException if e.getSQLState == "X0Y32" => // table exists } finally { - conn.close + conn.close() } } @@ -57,7 +57,7 @@ class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { () => { DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb") }, "SELECT DATA FROM FOO WHERE ? <= ID AND ID <= ?", 1, 100, 3, - (r: ResultSet) => { r.getInt(1) } ).cache + (r: ResultSet) => { r.getInt(1) } ).cache() assert(rdd.count === 100) assert(rdd.reduce(_+_) === 10100) 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 f9e994b13dfbc..447e38ec9dbd0 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -30,6 +30,19 @@ import org.apache.spark.SparkContext._ import org.apache.spark.{Partitioner, SharedSparkContext} class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { + test("aggregateByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 1), (3, 2), (5, 1), (5, 3)), 2) + + val sets = pairs.aggregateByKey(new HashSet[Int]())(_ += _, _ ++= _).collect() + assert(sets.size === 3) + val valuesFor1 = sets.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1)) + val valuesFor3 = sets.find(_._1 == 3).get._2 + assert(valuesFor3.toList.sorted === List(2)) + val valuesFor5 = sets.find(_._1 == 5).get._2 + assert(valuesFor5.toList.sorted === List(1, 3)) + } + test("groupByKey") { val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))) val groups = pairs.groupByKey().collect() @@ -119,28 +132,30 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { * relatively tight error bounds to check correctness of functionality rather than checking * whether the approximation conforms with the requested bound. */ - val relativeSD = 0.001 + val p = 20 + val sp = 0 + // When p = 20, the relative accuracy is about 0.001. So with high probability, the + // relative error should be smaller than the threshold 0.01 we use here. + val relativeSD = 0.01 // For each value i, there are i tuples with first element equal to i. // Therefore, the expected count for key i would be i. val stacked = (1 to 100).flatMap(i => (1 to i).map(j => (i, j))) val rdd1 = sc.parallelize(stacked) - val counted1 = rdd1.countApproxDistinctByKey(relativeSD).collect() - counted1.foreach{ - case(k, count) => assert(error(count, k) < relativeSD) - } + val counted1 = rdd1.countApproxDistinctByKey(p, sp).collect() + counted1.foreach { case (k, count) => assert(error(count, k) < relativeSD) } - val rnd = new Random() + val rnd = new Random(42) // The expected count for key num would be num val randStacked = (1 to 100).flatMap { i => - val num = rnd.nextInt % 500 + val num = rnd.nextInt() % 500 (1 to num).map(j => (num, j)) } val rdd2 = sc.parallelize(randStacked) - val counted2 = rdd2.countApproxDistinctByKey(relativeSD, 4).collect() - counted2.foreach{ - case(k, count) => assert(error(count, k) < relativeSD) + val counted2 = rdd2.countApproxDistinctByKey(relativeSD).collect() + counted2.foreach { case (k, count) => + assert(error(count, k) < relativeSD, s"${error(count, k)} < $relativeSD") } } @@ -225,21 +240,57 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val joined = rdd1.groupWith(rdd2).collect() assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (ArrayBuffer(1, 2), ArrayBuffer('x'))), - (2, (ArrayBuffer(1), ArrayBuffer('y', 'z'))), - (3, (ArrayBuffer(1), ArrayBuffer())), - (4, (ArrayBuffer(), ArrayBuffer('w'))) + val joinedSet = joined.map(x => (x._1, (x._2._1.toList, x._2._2.toList))).toSet + assert(joinedSet === Set( + (1, (List(1, 2), List('x'))), + (2, (List(1), List('y', 'z'))), + (3, (List(1), List())), + (4, (List(), List('w'))) + )) + } + + test("groupWith3") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val rdd3 = sc.parallelize(Array((1, 'a'), (3, 'b'), (4, 'c'), (4, 'd'))) + val joined = rdd1.groupWith(rdd2, rdd3).collect() + assert(joined.size === 4) + val joinedSet = joined.map(x => (x._1, + (x._2._1.toList, x._2._2.toList, x._2._3.toList))).toSet + assert(joinedSet === Set( + (1, (List(1, 2), List('x'), List('a'))), + (2, (List(1), List('y', 'z'), List())), + (3, (List(1), List(), List('b'))), + (4, (List(), List('w'), List('c', 'd'))) + )) + } + + test("groupWith4") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val rdd3 = sc.parallelize(Array((1, 'a'), (3, 'b'), (4, 'c'), (4, 'd'))) + val rdd4 = sc.parallelize(Array((2, '@'))) + val joined = rdd1.groupWith(rdd2, rdd3, rdd4).collect() + assert(joined.size === 4) + val joinedSet = joined.map(x => (x._1, + (x._2._1.toList, x._2._2.toList, x._2._3.toList, x._2._4.toList))).toSet + assert(joinedSet === Set( + (1, (List(1, 2), List('x'), List('a'), List())), + (2, (List(1), List('y', 'z'), List(), List('@'))), + (3, (List(1), List(), List('b'), List())), + (4, (List(), List('w'), List('c', 'd'), List())) )) } test("zero-partition RDD") { val emptyDir = Files.createTempDir() + emptyDir.deleteOnExit() val file = sc.textFile(emptyDir.getAbsolutePath) assert(file.partitions.size == 0) assert(file.collect().toList === Nil) // Test that a shuffle on the file works, because this used to be a bug assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) + emptyDir.delete() } test("keys and values") { @@ -447,4 +498,3 @@ class ConfigTestFormat() extends FakeFormat() with Configurable { super.getRecordWriter(p1) } } - 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 a4381a8b974df..1b112f1a41ca9 100644 --- a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala @@ -34,14 +34,14 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices(1).mkString(",") === "2") assert(slices(2).mkString(",") === "3") } - + test("one slice") { val data = Array(1, 2, 3) val slices = ParallelCollectionRDD.slice(data, 1) assert(slices.size === 1) assert(slices(0).mkString(",") === "1,2,3") } - + test("equal slices") { val data = Array(1, 2, 3, 4, 5, 6, 7, 8, 9) val slices = ParallelCollectionRDD.slice(data, 3) @@ -50,7 +50,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices(1).mkString(",") === "4,5,6") assert(slices(2).mkString(",") === "7,8,9") } - + test("non-equal slices") { val data = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) val slices = ParallelCollectionRDD.slice(data, 3) @@ -77,14 +77,14 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices(1).mkString(",") === (33 to 66).mkString(",")) assert(slices(2).mkString(",") === (67 to 100).mkString(",")) } - + test("empty data") { val data = new Array[Int](0) val slices = ParallelCollectionRDD.slice(data, 5) assert(slices.size === 5) for (slice <- slices) assert(slice.size === 0) } - + test("zero slices") { val data = Array(1, 2, 3) intercept[IllegalArgumentException] { ParallelCollectionRDD.slice(data, 0) } @@ -94,7 +94,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = Array(1, 2, 3) intercept[IllegalArgumentException] { ParallelCollectionRDD.slice(data, -5) } } - + test("exclusive ranges sliced into ranges") { val data = 1 until 100 val slices = ParallelCollectionRDD.slice(data, 3) @@ -102,7 +102,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices.map(_.size).reduceLeft(_+_) === 99) assert(slices.forall(_.isInstanceOf[Range])) } - + test("inclusive ranges sliced into ranges") { val data = 1 to 100 val slices = ParallelCollectionRDD.slice(data, 3) @@ -111,6 +111,24 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices.forall(_.isInstanceOf[Range])) } + test("identical slice sizes between Range and NumericRange") { + val r = ParallelCollectionRDD.slice(1 to 7, 4) + val nr = ParallelCollectionRDD.slice(1L to 7L, 4) + assert(r.size === 4) + for (i <- 0 until r.size) { + assert(r(i).size === nr(i).size) + } + } + + test("identical slice sizes between List and NumericRange") { + val r = ParallelCollectionRDD.slice(List(1, 2), 4) + val nr = ParallelCollectionRDD.slice(1L to 2L, 4) + assert(r.size === 4) + for (i <- 0 until r.size) { + assert(r(i).size === nr(i).size) + } + } + test("large ranges don't overflow") { val N = 100 * 1000 * 1000 val data = 0 until N @@ -124,7 +142,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(range.step === 1, "slice " + i + " step") } } - + test("random array tests") { val gen = for { d <- arbitrary[List[Int]] @@ -141,7 +159,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { } check(prop) } - + test("random exclusive range tests") { val gen = for { a <- Gen.choose(-100, 100) @@ -177,7 +195,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { } check(prop) } - + test("exclusive ranges of longs") { val data = 1L until 100L val slices = ParallelCollectionRDD.slice(data, 3) @@ -185,7 +203,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices.map(_.size).reduceLeft(_+_) === 99) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } - + test("inclusive ranges of longs") { val data = 1L to 100L val slices = ParallelCollectionRDD.slice(data, 3) @@ -193,7 +211,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices.map(_.size).reduceLeft(_+_) === 100) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } - + test("exclusive ranges of doubles") { val data = 1.0 until 100.0 by 1.0 val slices = ParallelCollectionRDD.slice(data, 3) @@ -201,7 +219,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices.map(_.size).reduceLeft(_+_) === 99) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } - + test("inclusive ranges of doubles") { val data = 1.0 to 100.0 by 1.0 val slices = ParallelCollectionRDD.slice(data, 3) 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 d6b5fdc7984b4..0e5625b7645d5 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -17,22 +17,23 @@ package org.apache.spark.rdd -import scala.collection.mutable.HashMap -import scala.collection.parallel.mutable +import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.reflect.ClassTag import org.scalatest.FunSuite -import org.scalatest.concurrent.Timeouts._ -import org.scalatest.time.{Millis, Span} import org.apache.spark._ import org.apache.spark.SparkContext._ -import org.apache.spark.rdd._ +import org.apache.spark.util.Utils + +import org.apache.spark.rdd.RDDSuiteUtils._ class RDDSuite extends FunSuite with SharedSparkContext { test("basic operations") { val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) assert(nums.collect().toList === List(1, 2, 3, 4)) + assert(nums.toLocalIterator.toList === List(1, 2, 3, 4)) val dups = sc.makeRDD(Array(1, 1, 2, 2, 3, 3, 4, 4), 2) assert(dups.distinct().count() === 4) assert(dups.distinct.count === 4) // Can distinct and count be called without parentheses? @@ -67,6 +68,13 @@ class RDDSuite extends FunSuite with SharedSparkContext { } } + test("serialization") { + val empty = new EmptyRDD[Int](sc) + val serial = Utils.serialize(empty) + val deserial: EmptyRDD[Int] = Utils.deserialize(serial) + assert(!deserial.toString().isEmpty()) + } + test("countApproxDistinct") { def error(est: Long, size: Long) = math.abs(est - size) / size.toDouble @@ -74,10 +82,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { val size = 100 val uniformDistro = for (i <- 1 to 100000) yield i % size val simpleRdd = sc.makeRDD(uniformDistro) - assert(error(simpleRdd.countApproxDistinct(0.2), size) < 0.2) - assert(error(simpleRdd.countApproxDistinct(0.05), size) < 0.05) - assert(error(simpleRdd.countApproxDistinct(0.01), size) < 0.01) - assert(error(simpleRdd.countApproxDistinct(0.001), size) < 0.001) + assert(error(simpleRdd.countApproxDistinct(4, 0), size) < 0.4) + assert(error(simpleRdd.countApproxDistinct(8, 0), size) < 0.1) } test("SparkContext.union") { @@ -152,7 +158,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { if (shouldFail) { throw new Exception("injected failure") } else { - return Array(1, 2, 3, 4).iterator + Array(1, 2, 3, 4).iterator } } }.cache() @@ -203,6 +209,39 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(repartitioned2.collect().toSet === (1 to 1000).toSet) } + test("repartitioned RDDs perform load balancing") { + // Coalesce partitions + val input = Array.fill(1000)(1) + val initialPartitions = 10 + val data = sc.parallelize(input, initialPartitions) + + val repartitioned1 = data.repartition(2) + assert(repartitioned1.partitions.size == 2) + val partitions1 = repartitioned1.glom().collect() + // some noise in balancing is allowed due to randomization + assert(math.abs(partitions1(0).length - 500) < initialPartitions) + assert(math.abs(partitions1(1).length - 500) < initialPartitions) + assert(repartitioned1.collect() === input) + + def testSplitPartitions(input: Seq[Int], initialPartitions: Int, finalPartitions: Int) { + val data = sc.parallelize(input, initialPartitions) + val repartitioned = data.repartition(finalPartitions) + assert(repartitioned.partitions.size === finalPartitions) + val partitions = repartitioned.glom().collect() + // assert all elements are present + assert(repartitioned.collect().sortWith(_ > _).toSeq === input.toSeq.sortWith(_ > _).toSeq) + // assert no bucket is overloaded + for (partition <- partitions) { + val avg = input.size / finalPartitions + val maxPossible = avg + initialPartitions + assert(partition.length <= maxPossible) + } + } + + testSplitPartitions(Array.fill(100)(1), 10, 20) + testSplitPartitions(Array.fill(10000)(1) ++ Array.fill(10000)(2), 20, 100) + } + test("coalesced RDDs") { val data = sc.parallelize(1 to 10, 10) @@ -236,8 +275,9 @@ class RDDSuite extends FunSuite with SharedSparkContext { // we can optionally shuffle to keep the upstream parallel val coalesced5 = data.coalesce(1, shuffle = true) - assert(coalesced5.dependencies.head.rdd.dependencies.head.rdd.asInstanceOf[ShuffledRDD[_, _, _]] != - null) + val isEquals = coalesced5.dependencies.head.rdd.dependencies.head.rdd. + asInstanceOf[ShuffledRDD[_, _, _]] != null + assert(isEquals) // when shuffling, we can increase the number of partitions val coalesced6 = data.coalesce(20, shuffle = true) @@ -273,37 +313,42 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("coalesced RDDs with locality, large scale (10K partitions)") { // large scale experiment import collection.mutable - val rnd = scala.util.Random val partitions = 10000 val numMachines = 50 val machines = mutable.ListBuffer[String]() - (1 to numMachines).foreach(machines += "m"+_) - - val blocks = (1 to partitions).map(i => - { (i, Array.fill(3)(machines(rnd.nextInt(machines.size))).toList) } ) - - val data2 = sc.makeRDD(blocks) - val coalesced2 = data2.coalesce(numMachines*2) - - // test that you get over 90% locality in each group - val minLocality = coalesced2.partitions - .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) - .foldLeft(1.0)((perc, loc) => math.min(perc,loc)) - assert(minLocality >= 0.90, "Expected 90% locality but got " + (minLocality*100.0).toInt + "%") + (1 to numMachines).foreach(machines += "m" + _) + val rnd = scala.util.Random + for (seed <- 1 to 5) { + rnd.setSeed(seed) - // test that the groups are load balanced with 100 +/- 20 elements in each - val maxImbalance = coalesced2.partitions - .map(part => part.asInstanceOf[CoalescedRDDPartition].parents.size) - .foldLeft(0)((dev, curr) => math.max(math.abs(100-curr),dev)) - assert(maxImbalance <= 20, "Expected 100 +/- 20 per partition, but got " + maxImbalance) + val blocks = (1 to partitions).map { i => + (i, Array.fill(3)(machines(rnd.nextInt(machines.size))).toList) + } - val data3 = sc.makeRDD(blocks).map(i => i*2) // derived RDD to test *current* pref locs - val coalesced3 = data3.coalesce(numMachines*2) - val minLocality2 = coalesced3.partitions - .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) - .foldLeft(1.0)((perc, loc) => math.min(perc,loc)) - assert(minLocality2 >= 0.90, "Expected 90% locality for derived RDD but got " + - (minLocality2*100.0).toInt + "%") + val data2 = sc.makeRDD(blocks) + val coalesced2 = data2.coalesce(numMachines * 2) + + // test that you get over 90% locality in each group + val minLocality = coalesced2.partitions + .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) + .foldLeft(1.0)((perc, loc) => math.min(perc, loc)) + assert(minLocality >= 0.90, "Expected 90% locality but got " + + (minLocality * 100.0).toInt + "%") + + // test that the groups are load balanced with 100 +/- 20 elements in each + val maxImbalance = coalesced2.partitions + .map(part => part.asInstanceOf[CoalescedRDDPartition].parents.size) + .foldLeft(0)((dev, curr) => math.max(math.abs(100 - curr), dev)) + assert(maxImbalance <= 20, "Expected 100 +/- 20 per partition, but got " + maxImbalance) + + val data3 = sc.makeRDD(blocks).map(i => i * 2) // derived RDD to test *current* pref locs + val coalesced3 = data3.coalesce(numMachines * 2) + val minLocality2 = coalesced3.partitions + .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) + .foldLeft(1.0)((perc, loc) => math.min(perc, loc)) + assert(minLocality2 >= 0.90, "Expected 90% locality for derived RDD but got " + + (minLocality2 * 100.0).toInt + "%") + } } test("zipped RDDs") { @@ -315,6 +360,10 @@ class RDDSuite extends FunSuite with SharedSparkContext { intercept[IllegalArgumentException] { nums.zip(sc.parallelize(1 to 4, 1)).collect() } + + intercept[SparkException] { + nums.zip(sc.parallelize(1 to 5, 2)).collect() + } } test("partition pruning") { @@ -377,8 +426,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { val prng42 = new Random(42) val prng43 = new Random(43) Array(1, 2, 3, 4, 5, 6).filter{i => - if (i < 4) 0 == prng42.nextInt(3) - else 0 == prng43.nextInt(3)} + if (i < 4) 0 == prng42.nextInt(3) else 0 == prng43.nextInt(3) + } } assert(sample.size === checkSample.size) for (i <- 0 until sample.size) assert(sample(i) === checkSample(i)) @@ -458,36 +507,56 @@ class RDDSuite extends FunSuite with SharedSparkContext { } test("takeSample") { - val data = sc.parallelize(1 to 100, 2) - + val n = 1000000 + val data = sc.parallelize(1 to n, 2) + + for (num <- List(5, 20, 100)) { + val sample = data.takeSample(withReplacement=false, num=num) + assert(sample.size === num) // Got exactly num elements + assert(sample.toSet.size === num) // Elements are distinct + assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") + } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement=false, 20, seed) assert(sample.size === 20) // Got exactly 20 elements assert(sample.toSet.size === 20) // Elements are distinct - assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") } for (seed <- 1 to 5) { - val sample = data.takeSample(withReplacement=false, 200, seed) + val sample = data.takeSample(withReplacement=false, 100, seed) assert(sample.size === 100) // Got only 100 elements assert(sample.toSet.size === 100) // Elements are distinct - assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement=true, 20, seed) assert(sample.size === 20) // Got exactly 20 elements - assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") + } + { + val sample = data.takeSample(withReplacement=true, num=20) + assert(sample.size === 20) // Got exactly 100 elements + assert(sample.toSet.size <= 20, "sampling with replacement returned all distinct elements") + assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") + } + { + val sample = data.takeSample(withReplacement=true, num=n) + assert(sample.size === n) // Got exactly 100 elements + // Chance of getting all distinct elements is astronomically low, so test we got < 100 + assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") + assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") } for (seed <- 1 to 5) { - val sample = data.takeSample(withReplacement=true, 100, seed) - assert(sample.size === 100) // Got exactly 100 elements + val sample = data.takeSample(withReplacement=true, n, seed) + assert(sample.size === n) // Got exactly 100 elements // Chance of getting all distinct elements is astronomically low, so test we got < 100 - assert(sample.toSet.size < 100, "sampling with replacement returned all distinct elements") + assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") } for (seed <- 1 to 5) { - val sample = data.takeSample(withReplacement=true, 200, seed) - assert(sample.size === 200) // Got exactly 200 elements + val sample = data.takeSample(withReplacement=true, 2 * n, seed) + assert(sample.size === 2 * n) // Got exactly 200 elements // Chance of getting all distinct elements is still quite low, so test we got < 100 - assert(sample.toSet.size < 100, "sampling with replacement returned all distinct elements") + assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") } } @@ -518,14 +587,63 @@ class RDDSuite extends FunSuite with SharedSparkContext { } } + test("sortByKey") { + val data = sc.parallelize(Seq("5|50|A","4|60|C", "6|40|B")) + + val col1 = Array("4|60|C", "5|50|A", "6|40|B") + val col2 = Array("6|40|B", "5|50|A", "4|60|C") + val col3 = Array("5|50|A", "6|40|B", "4|60|C") + + assert(data.sortBy(_.split("\\|")(0)).collect() === col1) + assert(data.sortBy(_.split("\\|")(1)).collect() === col2) + assert(data.sortBy(_.split("\\|")(2)).collect() === col3) + } + + test("sortByKey ascending parameter") { + val data = sc.parallelize(Seq("5|50|A","4|60|C", "6|40|B")) + + val asc = Array("4|60|C", "5|50|A", "6|40|B") + val desc = Array("6|40|B", "5|50|A", "4|60|C") + + assert(data.sortBy(_.split("\\|")(0), true).collect() === asc) + assert(data.sortBy(_.split("\\|")(0), false).collect() === desc) + } + + test("sortByKey with explicit ordering") { + val data = sc.parallelize(Seq("Bob|Smith|50", + "Jane|Smith|40", + "Thomas|Williams|30", + "Karen|Williams|60")) + + val ageOrdered = Array("Thomas|Williams|30", + "Jane|Smith|40", + "Bob|Smith|50", + "Karen|Williams|60") + + // last name, then first name + val nameOrdered = Array("Bob|Smith|50", + "Jane|Smith|40", + "Karen|Williams|60", + "Thomas|Williams|30") + + val parse = (s: String) => { + val split = s.split("\\|") + Person(split(0), split(1), split(2).toInt) + } + + import scala.reflect.classTag + assert(data.sortBy(parse, true, 2)(AgeOrdering, classTag[Person]).collect() === ageOrdered) + assert(data.sortBy(parse, true, 2)(NameOrdering, classTag[Person]).collect() === nameOrdered) + } + test("intersection") { val all = sc.parallelize(1 to 10) val evens = sc.parallelize(2 to 10 by 2) val intersection = Array(2, 4, 6, 8, 10) // intersection is commutative - assert(all.intersection(evens).collect.sorted === intersection) - assert(evens.intersection(all).collect.sorted === intersection) + assert(all.intersection(evens).collect().sorted === intersection) + assert(evens.intersection(all).collect().sorted === intersection) } test("intersection strips duplicates in an input") { @@ -533,8 +651,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { val b = sc.parallelize(Seq(1,1,2,3)) val intersection = Array(1,2,3) - assert(a.intersection(b).collect.sorted === intersection) - assert(b.intersection(a).collect.sorted === intersection) + assert(a.intersection(b).collect().sorted === intersection) + assert(b.intersection(a).collect().sorted === intersection) } test("zipWithIndex") { @@ -562,4 +680,144 @@ class RDDSuite extends FunSuite with SharedSparkContext { val ids = ranked.map(_._1).distinct().collect() assert(ids.length === n) } + + test("getNarrowAncestors") { + val rdd1 = sc.parallelize(1 to 100, 4) + val rdd2 = rdd1.filter(_ % 2 == 0).map(_ + 1) + val rdd3 = rdd2.map(_ - 1).filter(_ < 50).map(i => (i, i)) + val rdd4 = rdd3.reduceByKey(_ + _) + val rdd5 = rdd4.mapValues(_ + 1).mapValues(_ + 2).mapValues(_ + 3) + val ancestors1 = rdd1.getNarrowAncestors + val ancestors2 = rdd2.getNarrowAncestors + val ancestors3 = rdd3.getNarrowAncestors + val ancestors4 = rdd4.getNarrowAncestors + val ancestors5 = rdd5.getNarrowAncestors + + // Simple dependency tree with a single branch + assert(ancestors1.size === 0) + assert(ancestors2.size === 2) + assert(ancestors2.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 1) + assert(ancestors2.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors3.size === 5) + assert(ancestors3.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 1) + assert(ancestors3.count(_.isInstanceOf[FilteredRDD[_]]) === 2) + assert(ancestors3.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) + + // Any ancestors before the shuffle are not considered + assert(ancestors4.size === 1) + assert(ancestors4.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 1) + assert(ancestors5.size === 4) + assert(ancestors5.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 1) + assert(ancestors5.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 1) + assert(ancestors5.count(_.isInstanceOf[MappedValuesRDD[_, _, _]]) === 2) + } + + test("getNarrowAncestors with multiple parents") { + val rdd1 = sc.parallelize(1 to 100, 5) + val rdd2 = sc.parallelize(1 to 200, 10).map(_ + 1) + val rdd3 = sc.parallelize(1 to 300, 15).filter(_ > 50) + val rdd4 = rdd1.map(i => (i, i)) + val rdd5 = rdd2.map(i => (i, i)) + val rdd6 = sc.union(rdd1, rdd2) + val rdd7 = sc.union(rdd1, rdd2, rdd3) + val rdd8 = sc.union(rdd6, rdd7) + val rdd9 = rdd4.join(rdd5) + val ancestors6 = rdd6.getNarrowAncestors + val ancestors7 = rdd7.getNarrowAncestors + val ancestors8 = rdd8.getNarrowAncestors + val ancestors9 = rdd9.getNarrowAncestors + + // Simple dependency tree with multiple branches + assert(ancestors6.size === 3) + assert(ancestors6.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 2) + assert(ancestors6.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) + assert(ancestors7.size === 5) + assert(ancestors7.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 3) + assert(ancestors7.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) + assert(ancestors7.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + + // Dependency tree with duplicate nodes (e.g. rdd1 should not be reported twice) + assert(ancestors8.size === 7) + assert(ancestors8.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) + assert(ancestors8.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors8.count(_.isInstanceOf[UnionRDD[_]]) === 2) + assert(ancestors8.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 3) + assert(ancestors8.count(_ == rdd1) === 1) + assert(ancestors8.count(_ == rdd2) === 1) + assert(ancestors8.count(_ == rdd3) === 1) + + // Any ancestors before the shuffle are not considered + assert(ancestors9.size === 2) + assert(ancestors9.count(_.isInstanceOf[CoGroupedRDD[_]]) === 1) + assert(ancestors9.count(_.isInstanceOf[MappedValuesRDD[_, _, _]]) === 1) + } + + /** + * This tests for the pathological condition in which the RDD dependency graph is cyclical. + * + * Since RDD is part of the public API, applications may actually implement RDDs that allow + * such graphs to be constructed. In such cases, getNarrowAncestor should not simply hang. + */ + test("getNarrowAncestors with cycles") { + val rdd1 = new CyclicalDependencyRDD[Int] + val rdd2 = new CyclicalDependencyRDD[Int] + val rdd3 = new CyclicalDependencyRDD[Int] + val rdd4 = rdd3.map(_ + 1).filter(_ > 10).map(_ + 2).filter(_ % 5 > 1) + val rdd5 = rdd4.map(_ + 2).filter(_ > 20) + val rdd6 = sc.union(rdd1, rdd2, rdd3).map(_ + 4).union(rdd5).union(rdd4) + + // Simple cyclical dependency + rdd1.addDependency(new OneToOneDependency[Int](rdd2)) + rdd2.addDependency(new OneToOneDependency[Int](rdd1)) + val ancestors1 = rdd1.getNarrowAncestors + val ancestors2 = rdd2.getNarrowAncestors + assert(ancestors1.size === 1) + assert(ancestors1.count(_ == rdd2) === 1) + assert(ancestors1.count(_ == rdd1) === 0) + assert(ancestors2.size === 1) + assert(ancestors2.count(_ == rdd1) === 1) + assert(ancestors2.count(_ == rdd2) === 0) + + // Cycle involving a longer chain + rdd3.addDependency(new OneToOneDependency[Int](rdd4)) + val ancestors3 = rdd3.getNarrowAncestors + val ancestors4 = rdd4.getNarrowAncestors + assert(ancestors3.size === 4) + assert(ancestors3.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) + assert(ancestors3.count(_.isInstanceOf[FilteredRDD[_]]) === 2) + assert(ancestors3.count(_ == rdd3) === 0) + assert(ancestors4.size === 4) + assert(ancestors4.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) + assert(ancestors4.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors4.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 1) + assert(ancestors4.count(_ == rdd3) === 1) + assert(ancestors4.count(_ == rdd4) === 0) + + // Cycles that do not involve the root + val ancestors5 = rdd5.getNarrowAncestors + assert(ancestors5.size === 6) + assert(ancestors5.count(_.isInstanceOf[MappedRDD[_, _]]) === 3) + assert(ancestors5.count(_.isInstanceOf[FilteredRDD[_]]) === 2) + assert(ancestors5.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 1) + assert(ancestors4.count(_ == rdd3) === 1) + + // Complex cyclical dependency graph (combination of all of the above) + val ancestors6 = rdd6.getNarrowAncestors + assert(ancestors6.size === 12) + assert(ancestors6.count(_.isInstanceOf[UnionRDD[_]]) === 2) + assert(ancestors6.count(_.isInstanceOf[MappedRDD[_, _]]) === 4) + assert(ancestors6.count(_.isInstanceOf[FilteredRDD[_]]) === 3) + assert(ancestors6.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 3) + } + + /** A contrived RDD that allows the manual addition of dependencies after creation. */ + private class CyclicalDependencyRDD[T: ClassTag] extends RDD[T](sc, Nil) { + private val mutableDependencies: ArrayBuffer[Dependency[_]] = ArrayBuffer.empty + override def compute(p: Partition, c: TaskContext): Iterator[T] = Iterator.empty + override def getPartitions: Array[Partition] = Array.empty + override def getDependencies: Seq[Dependency[_]] = mutableDependencies + def addDependency(dep: Dependency[_]) { + mutableDependencies += dep + } + } } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuiteUtils.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuiteUtils.scala new file mode 100644 index 0000000000000..4762fc17855ce --- /dev/null +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuiteUtils.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.rdd + +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 + } + + object NameOrdering extends Ordering[Person] { + def compare(a:Person, b:Person) = + implicitly[Ordering[Tuple2[String,String]]].compare((a.last, a.first), (b.last, b.first)) + } +} diff --git a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala index d0619559bb457..656917628f7a8 100644 --- a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala @@ -18,12 +18,12 @@ package org.apache.spark.rdd import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers +import org.scalatest.Matchers import org.apache.spark.{Logging, SharedSparkContext} import org.apache.spark.SparkContext._ -class SortingSuite extends FunSuite with SharedSparkContext with ShouldMatchers with Logging { +class SortingSuite extends FunSuite with SharedSparkContext with Matchers with Logging { test("sortByKey") { val pairs = sc.parallelize(Array((1, 0), (2, 0), (0, 0), (3, 0)), 2) diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala new file mode 100644 index 0000000000000..efef9d26dadca --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.{LocalSparkContext, SparkConf, SparkException, SparkContext} +import org.apache.spark.util.{SerializableBuffer, AkkaUtils} + +import org.scalatest.FunSuite + +class CoarseGrainedSchedulerBackendSuite extends FunSuite with LocalSparkContext { + + test("serialized task larger than akka frame size") { + val conf = new SparkConf + conf.set("spark.akka.frameSize","1") + conf.set("spark.default.parallelism","1") + sc = new SparkContext("local-cluster[2 , 1 , 512]", "test", conf) + val frameSize = AkkaUtils.maxFrameSizeBytes(sc.conf) + val buffer = new SerializableBuffer(java.nio.ByteBuffer.allocate(2 * frameSize)) + val larger = sc.parallelize(Seq(buffer)) + val thrown = intercept[SparkException] { + larger.collect() + } + assert(thrown.getMessage.contains("Consider using broadcast variables for large values")) + val smaller = sc.parallelize(1 to 4).collect() + assert(smaller.size === 4) + } + +} 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 ce567b0cde85d..45368328297d3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -18,31 +18,36 @@ package org.apache.spark.scheduler import scala.Tuple2 -import scala.collection.mutable.{HashMap, Map} +import scala.collection.mutable.{HashSet, HashMap, Map} +import scala.language.reflectiveCalls -import org.scalatest.{BeforeAndAfter, FunSuite} +import akka.actor._ +import akka.testkit.{ImplicitSender, TestKit, TestActorRef} +import org.scalatest.{BeforeAndAfter, FunSuiteLike} import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} +import org.apache.spark.util.CallSite + +class BuggyDAGEventProcessActor extends Actor { + val state = 0 + def receive = { + case _ => throw new SparkException("error") + } +} + +class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with FunSuiteLike + with ImplicitSender with BeforeAndAfter with LocalSparkContext { -/** - * Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler - * rather than spawning an event loop thread as happens in the real code. They use EasyMock - * to mock out two classes that DAGScheduler interacts with: TaskScheduler (to which TaskSets are - * submitted) and BlockManagerMaster (from which cache locations are retrieved and to which dead - * host notifications are sent). In addition, tests may check for side effects on a non-mocked - * MapOutputTracker instance. - * - * Tests primarily consist of running DAGScheduler#processEvent and - * DAGScheduler#submitWaitingStages (via test utility functions like runEvent or respondToTaskSet) - * and capturing the resulting TaskSets from the mock TaskScheduler. - */ -class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { val conf = new SparkConf /** Set of TaskSets the DAGScheduler has requested executed. */ val taskSets = scala.collection.mutable.Buffer[TaskSet]() + + /** Stages for which the DAGScheduler has called TaskScheduler.cancelTasks(). */ + val cancelledStages = new HashSet[Int]() + val taskScheduler = new TaskScheduler() { override def rootPool: Pool = null override def schedulingMode: SchedulingMode = SchedulingMode.NONE @@ -53,13 +58,31 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) taskSets += taskSet } - override def cancelTasks(stageId: Int) {} + override def cancelTasks(stageId: Int, interruptThread: Boolean) { + cancelledStages += stageId + } override def setDAGScheduler(dagScheduler: DAGScheduler) = {} override def defaultParallelism() = 2 } + /** Length of time to wait while draining listener events. */ + val WAIT_TIMEOUT_MILLIS = 10000 + val sparkListener = new SparkListener() { + val successfulStages = new HashSet[Int]() + val failedStages = new HashSet[Int]() + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { + val stageInfo = stageCompleted.stageInfo + if (stageInfo.failureReason.isEmpty) { + successfulStages += stageInfo.stageId + } else { + failedStages += stageInfo.stageId + } + } + } + var mapOutputTracker: MapOutputTrackerMaster = null var scheduler: DAGScheduler = null + var dagEventProcessTestActor: TestActorRef[DAGSchedulerEventProcessActor] = null /** * Set of cache locations to return from our mock BlockManagerMaster. @@ -83,18 +106,23 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont /** The list of results that DAGScheduler has collected. */ val results = new HashMap[Int, Any]() var failure: Exception = _ - val listener = new JobListener() { + val jobListener = new JobListener() { override def taskSucceeded(index: Int, result: Any) = results.put(index, result) override def jobFailed(exception: Exception) = { failure = exception } } before { sc = new SparkContext("local", "DAGSchedulerSuite") + sparkListener.successfulStages.clear() + sparkListener.failedStages.clear() + sc.addSparkListener(sparkListener) taskSets.clear() + cancelledStages.clear() cacheLocations.clear() results.clear() mapOutputTracker = new MapOutputTrackerMaster(conf) scheduler = new DAGScheduler( + sc, taskScheduler, sc.listenerBus, mapOutputTracker, @@ -105,10 +133,13 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont runLocallyWithinThread(job) } } + dagEventProcessTestActor = TestActorRef[DAGSchedulerEventProcessActor]( + Props(classOf[DAGSchedulerEventProcessActor], scheduler))(system) } - after { - scheduler.stop() + override def afterAll() { + super.afterAll() + TestKit.shutdownActorSystem(system) } /** @@ -152,8 +183,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont * DAGScheduler event loop. */ private def runEvent(event: DAGSchedulerEvent) { - assert(!scheduler.processEvent(event)) - scheduler.submitWaitingStages() + dagEventProcessTestActor.receive(event) } /** @@ -174,15 +204,16 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont } } - /** Sends the rdd to the scheduler for scheduling. */ + /** Sends the rdd to the scheduler for scheduling and returns the job id. */ private def submit( rdd: RDD[_], partitions: Array[Int], func: (TaskContext, Iterator[_]) => _ = jobComputeFunc, allowLocal: Boolean = false, - listener: JobListener = listener) { + listener: JobListener = jobListener): Int = { val jobId = scheduler.nextJobId.getAndIncrement() - runEvent(JobSubmitted(jobId, rdd, func, partitions, allowLocal, null, listener)) + runEvent(JobSubmitted(jobId, rdd, func, partitions, allowLocal, CallSite("", ""), listener)) + jobId } /** Sends TaskSetFailed to the scheduler. */ @@ -190,20 +221,23 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont runEvent(TaskSetFailed(taskSet, message)) } + /** Sends JobCancelled to the DAG scheduler. */ + private def cancel(jobId: Int) { + runEvent(JobCancelled(jobId)) + } + test("zero split job") { - val rdd = makeRdd(0, Nil) var numResults = 0 val fakeListener = new JobListener() { override def taskSucceeded(partition: Int, value: Any) = numResults += 1 override def jobFailed(exception: Exception) = throw exception } - submit(rdd, Array(), listener = fakeListener) + submit(makeRdd(0, Nil), Array(), listener = fakeListener) assert(numResults === 0) } test("run trivial job") { - val rdd = makeRdd(1, Nil) - submit(rdd, Array(0)) + submit(makeRdd(1, Nil), Array(0)) complete(taskSets(0), List((Success, 42))) assert(results === Map(0 -> 42)) assertDataStructuresEmpty @@ -218,11 +252,25 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont override def toString = "DAGSchedulerSuite Local RDD" } val jobId = scheduler.nextJobId.getAndIncrement() - runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, null, listener)) + runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, CallSite("", ""), jobListener)) assert(results === Map(0 -> 42)) assertDataStructuresEmpty } + test("local job oom") { + val rdd = new MyRDD(sc, Nil) { + override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = + throw new java.lang.OutOfMemoryError("test local job oom") + override def getPartitions = Array( new Partition { override def index = 0 } ) + override def getPreferredLocations(split: Partition) = Nil + override def toString = "DAGSchedulerSuite Local RDD" + } + val jobId = scheduler.nextJobId.getAndIncrement() + runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, CallSite("", ""), jobListener)) + assert(results.size == 0) + assertDataStructuresEmpty + } + test("run trivial job w/ dependency") { val baseRdd = makeRdd(1, Nil) val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd))) @@ -248,7 +296,21 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont test("trivial job failure") { submit(makeRdd(1, Nil), Array(0)) failed(taskSets(0), "some failure") - assert(failure.getMessage === "Job aborted: some failure") + assert(failure.getMessage === "Job aborted due to stage failure: some failure") + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.contains(0)) + assert(sparkListener.failedStages.size === 1) + assertDataStructuresEmpty + } + + test("trivial job cancellation") { + val rdd = makeRdd(1, Nil) + val jobId = submit(rdd, Array(0)) + cancel(jobId) + assert(failure.getMessage === s"Job $jobId cancelled ") + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.contains(0)) + assert(sparkListener.failedStages.size === 1) assertDataStructuresEmpty } @@ -323,6 +385,82 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assertDataStructuresEmpty } + test("run shuffle with map stage failure") { + val shuffleMapRdd = makeRdd(2, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) + val reduceRdd = makeRdd(2, List(shuffleDep)) + submit(reduceRdd, Array(0, 1)) + + // Fail the map stage. This should cause the entire job to fail. + val stageFailureMessage = "Exception failure in map stage" + failed(taskSets(0), stageFailureMessage) + assert(failure.getMessage === s"Job aborted due to stage failure: $stageFailureMessage") + + // Listener bus should get told about the map stage failing, but not the reduce stage + // (since the reduce stage hasn't been started yet). + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.contains(1)) + assert(sparkListener.failedStages.size === 1) + + assertDataStructuresEmpty + } + + /** + * Makes sure that failures of stage used by multiple jobs are correctly handled. + * + * This test creates the following dependency graph: + * + * shuffleMapRdd1 shuffleMapRDD2 + * | \ | + * | \ | + * | \ | + * | \ | + * reduceRdd1 reduceRdd2 + * + * We start both shuffleMapRdds and then fail shuffleMapRdd1. As a result, the job listeners for + * reduceRdd1 and reduceRdd2 should both be informed that the job failed. shuffleMapRDD2 should + * also be cancelled, because it is only used by reduceRdd2 and reduceRdd2 cannot complete + * without shuffleMapRdd1. + */ + test("failure of stage used by two jobs") { + val shuffleMapRdd1 = makeRdd(2, Nil) + val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, null) + val shuffleMapRdd2 = makeRdd(2, Nil) + val shuffleDep2 = new ShuffleDependency(shuffleMapRdd2, null) + + val reduceRdd1 = makeRdd(2, List(shuffleDep1)) + val reduceRdd2 = makeRdd(2, List(shuffleDep1, shuffleDep2)) + + // We need to make our own listeners for this test, since by default submit uses the same + // listener for all jobs, and here we want to capture the failure for each job separately. + class FailureRecordingJobListener() extends JobListener { + var failureMessage: String = _ + override def taskSucceeded(index: Int, result: Any) {} + override def jobFailed(exception: Exception) = { failureMessage = exception.getMessage } + } + val listener1 = new FailureRecordingJobListener() + val listener2 = new FailureRecordingJobListener() + + submit(reduceRdd1, Array(0, 1), listener=listener1) + submit(reduceRdd2, Array(0, 1), listener=listener2) + + val stageFailureMessage = "Exception failure in map stage" + failed(taskSets(0), stageFailureMessage) + + assert(cancelledStages.contains(1)) + + // Make sure the listeners got told about both failed stages. + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.successfulStages.isEmpty) + assert(sparkListener.failedStages.contains(1)) + assert(sparkListener.failedStages.contains(3)) + assert(sparkListener.failedStages.size === 2) + + assert(listener1.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") + assert(listener2.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") + assertDataStructuresEmpty + } + test("run trivial shuffle with out-of-band failure and retry") { val shuffleMapRdd = makeRdd(2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) @@ -407,6 +545,18 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assertDataStructuresEmpty } + test("DAGSchedulerActorSupervisor closes the SparkContext when EventProcessActor crashes") { + val actorSystem = ActorSystem("test") + val supervisor = actorSystem.actorOf( + Props(classOf[DAGSchedulerActorSupervisor], scheduler), "dagSupervisor") + supervisor ! Props[BuggyDAGEventProcessActor] + val child = expectMsgType[ActorRef] + watch(child) + child ! "hi" + expectMsgPF(){ case Terminated(child) => () } + assert(scheduler.sc.dagScheduler === null) + } + /** * Assert that the supplied TaskSet has exactly the given hosts as its preferred locations. * Note that this checks only the host and not the executor ID. @@ -439,3 +589,4 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assert(scheduler.waitingStages.isEmpty) } } + diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala new file mode 100644 index 0000000000000..21e3db34b8b7a --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -0,0 +1,407 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 scala.collection.mutable +import scala.io.Source + +import com.google.common.io.Files +import org.apache.hadoop.fs.{FileStatus, Path} +import org.json4s.jackson.JsonMethods._ +import org.scalatest.{BeforeAndAfter, FunSuite} + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.{JsonProtocol, Utils} + +import java.io.File + +/** + * Test whether EventLoggingListener logs events properly. + * + * This tests whether EventLoggingListener actually creates special files while logging events, + * whether the parsing of these special files is correct, and whether the logged events can be + * read and deserialized into actual SparkListenerEvents. + */ +class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { + private val fileSystem = Utils.getHadoopFileSystem("/") + private val allCompressionCodecs = Seq[String]( + "org.apache.spark.io.LZFCompressionCodec", + "org.apache.spark.io.SnappyCompressionCodec" + ) + private var testDir: File = _ + private var logDirPath: Path = _ + + before { + testDir = Files.createTempDir() + testDir.deleteOnExit() + logDirPath = Utils.getFilePath(testDir, "spark-events") + } + + after { + Utils.deleteRecursively(testDir) + } + + test("Parse names of special files") { + testParsingFileName() + } + + test("Verify special files exist") { + testSpecialFilesExist() + } + + test("Verify special files exist with compression") { + allCompressionCodecs.foreach { codec => + testSpecialFilesExist(compressionCodec = Some(codec)) + } + } + + test("Parse event logging info") { + testParsingLogInfo() + } + + test("Parse event logging info with compression") { + allCompressionCodecs.foreach { codec => + testParsingLogInfo(compressionCodec = Some(codec)) + } + } + + test("Basic event logging") { + testEventLogging() + } + + test("Basic event logging with compression") { + allCompressionCodecs.foreach { codec => + testEventLogging(compressionCodec = Some(codec)) + } + } + + test("End-to-end event logging") { + testApplicationEventLogging() + } + + test("End-to-end event logging with compression") { + allCompressionCodecs.foreach { codec => + testApplicationEventLogging(compressionCodec = Some(codec)) + } + } + + + /* ----------------- * + * Actual test logic * + * ----------------- */ + + import EventLoggingListenerSuite._ + + /** + * Test whether names of special files are correctly identified and parsed. + */ + private def testParsingFileName() { + val logPrefix = EventLoggingListener.LOG_PREFIX + val sparkVersionPrefix = EventLoggingListener.SPARK_VERSION_PREFIX + val compressionCodecPrefix = EventLoggingListener.COMPRESSION_CODEC_PREFIX + val applicationComplete = EventLoggingListener.APPLICATION_COMPLETE + assert(EventLoggingListener.isEventLogFile(logPrefix + "0")) + assert(EventLoggingListener.isEventLogFile(logPrefix + "100")) + assert(EventLoggingListener.isEventLogFile(logPrefix + "ANYTHING")) + assert(EventLoggingListener.isSparkVersionFile(sparkVersionPrefix + "0.9.1")) + assert(EventLoggingListener.isSparkVersionFile(sparkVersionPrefix + "1.0.0")) + assert(EventLoggingListener.isSparkVersionFile(sparkVersionPrefix + "ANYTHING")) + assert(EventLoggingListener.isApplicationCompleteFile(applicationComplete)) + allCompressionCodecs.foreach { codec => + assert(EventLoggingListener.isCompressionCodecFile(compressionCodecPrefix + codec)) + } + + // Negatives + assert(!EventLoggingListener.isEventLogFile("The greatest man of all mankind")) + assert(!EventLoggingListener.isSparkVersionFile("Will never falter in the face of death!")) + assert(!EventLoggingListener.isCompressionCodecFile("Unless he chooses to leave behind")) + assert(!EventLoggingListener.isApplicationCompleteFile("The very treasure he calls Macbeth")) + + // Verify that parsing is correct + assert(EventLoggingListener.parseSparkVersion(sparkVersionPrefix + "1.0.0") === "1.0.0") + allCompressionCodecs.foreach { codec => + assert(EventLoggingListener.parseCompressionCodec(compressionCodecPrefix + codec) === codec) + } + } + + /** + * Test whether the special files produced by EventLoggingListener exist. + * + * There should be exactly one event log and one spark version file throughout the entire + * execution. If a compression codec is specified, then the compression codec file should + * also exist. Only after the application has completed does the test expect the application + * completed file to be present. + */ + private def testSpecialFilesExist(compressionCodec: Option[String] = None) { + + def assertFilesExist(logFiles: Array[FileStatus], loggerStopped: Boolean) { + val numCompressionCodecFiles = if (compressionCodec.isDefined) 1 else 0 + val numApplicationCompleteFiles = if (loggerStopped) 1 else 0 + assert(logFiles.size === 2 + numCompressionCodecFiles + numApplicationCompleteFiles) + assert(eventLogsExist(logFiles)) + assert(sparkVersionExists(logFiles)) + assert(compressionCodecExists(logFiles) === compressionCodec.isDefined) + assert(applicationCompleteExists(logFiles) === loggerStopped) + assertSparkVersionIsValid(logFiles) + compressionCodec.foreach { codec => + assertCompressionCodecIsValid(logFiles, codec) + } + } + + // Verify logging directory exists + val conf = getLoggingConf(logDirPath, compressionCodec) + val eventLogger = new EventLoggingListener("test", conf) + eventLogger.start() + val logPath = new Path(eventLogger.logDir) + assert(fileSystem.exists(logPath)) + val logDir = fileSystem.getFileStatus(logPath) + assert(logDir.isDir) + + // Verify special files are as expected before stop() + var logFiles = fileSystem.listStatus(logPath) + assert(logFiles != null) + assertFilesExist(logFiles, loggerStopped = false) + + // Verify special files are as expected after stop() + eventLogger.stop() + logFiles = fileSystem.listStatus(logPath) + assertFilesExist(logFiles, loggerStopped = true) + } + + /** + * Test whether EventLoggingListener correctly parses the correct information from the logs. + * + * This includes whether it returns the correct Spark version, compression codec (if any), + * and the application's completion status. + */ + private def testParsingLogInfo(compressionCodec: Option[String] = None) { + + def assertInfoCorrect(info: EventLoggingInfo, loggerStopped: Boolean) { + assert(info.logPaths.size > 0) + assert(info.sparkVersion === SparkContext.SPARK_VERSION) + assert(info.compressionCodec.isDefined === compressionCodec.isDefined) + info.compressionCodec.foreach { codec => + assert(compressionCodec.isDefined) + val expectedCodec = compressionCodec.get.split('.').last + assert(codec.getClass.getSimpleName === expectedCodec) + } + assert(info.applicationComplete === loggerStopped) + } + + // Verify that all information is correctly parsed before stop() + val conf = getLoggingConf(logDirPath, compressionCodec) + val eventLogger = new EventLoggingListener("test", conf) + eventLogger.start() + var eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) + assertInfoCorrect(eventLoggingInfo, loggerStopped = false) + + // Verify that all information is correctly parsed after stop() + eventLogger.stop() + eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) + assertInfoCorrect(eventLoggingInfo, loggerStopped = true) + } + + /** + * Test basic event logging functionality. + * + * This creates two simple events, posts them to the EventLoggingListener, and verifies that + * exactly these two events are logged in the expected file. + */ + private def testEventLogging(compressionCodec: Option[String] = None) { + val conf = getLoggingConf(logDirPath, compressionCodec) + val eventLogger = new EventLoggingListener("test", conf) + val listenerBus = new LiveListenerBus + val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", 125L, "Mickey") + val applicationEnd = SparkListenerApplicationEnd(1000L) + + // A comprehensive test on JSON de/serialization of all events is in JsonProtocolSuite + eventLogger.start() + listenerBus.start() + listenerBus.addListener(eventLogger) + listenerBus.postToAll(applicationStart) + listenerBus.postToAll(applicationEnd) + + // Verify file contains exactly the two events logged + val eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) + assert(eventLoggingInfo.logPaths.size > 0) + val lines = readFileLines(eventLoggingInfo.logPaths.head, eventLoggingInfo.compressionCodec) + assert(lines.size === 2) + assert(lines(0).contains("SparkListenerApplicationStart")) + assert(lines(1).contains("SparkListenerApplicationEnd")) + assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === applicationStart) + assert(JsonProtocol.sparkEventFromJson(parse(lines(1))) === applicationEnd) + eventLogger.stop() + } + + /** + * Test end-to-end event logging functionality in an application. + * This runs a simple Spark job and asserts that the expected events are logged when expected. + */ + private def testApplicationEventLogging(compressionCodec: Option[String] = None) { + val conf = getLoggingConf(logDirPath, compressionCodec) + val sc = new SparkContext("local", "test", conf) + assert(sc.eventLogger.isDefined) + val eventLogger = sc.eventLogger.get + val expectedLogDir = logDirPath.toString + assert(eventLogger.logDir.startsWith(expectedLogDir)) + + // Begin listening for events that trigger asserts + val eventExistenceListener = new EventExistenceListener(eventLogger) + sc.addSparkListener(eventExistenceListener) + + // Trigger asserts for whether the expected events are actually logged + sc.parallelize(1 to 10000).count() + sc.stop() + + // Ensure all asserts have actually been triggered + eventExistenceListener.assertAllCallbacksInvoked() + } + + /** + * Assert that all of the specified events are logged by the given EventLoggingListener. + */ + private def assertEventsExist(eventLogger: EventLoggingListener, events: Seq[String]) { + val eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) + assert(eventLoggingInfo.logPaths.size > 0) + val lines = readFileLines(eventLoggingInfo.logPaths.head, eventLoggingInfo.compressionCodec) + val eventSet = mutable.Set(events: _*) + lines.foreach { line => + eventSet.foreach { event => + if (line.contains(event)) { + val parsedEvent = JsonProtocol.sparkEventFromJson(parse(line)) + val eventType = Utils.getFormattedClassName(parsedEvent) + if (eventType == event) { + eventSet.remove(event) + } + } + } + } + assert(eventSet.isEmpty, "The following events are missing: " + eventSet.toSeq) + } + + /** + * Read all lines from the file specified by the given path. + * If a compression codec is specified, use it to read the file. + */ + private def readFileLines( + filePath: Path, + compressionCodec: Option[CompressionCodec]): Seq[String] = { + val fstream = fileSystem.open(filePath) + val cstream = + compressionCodec.map { codec => + codec.compressedInputStream(fstream) + }.getOrElse(fstream) + Source.fromInputStream(cstream).getLines().toSeq + } + + /** + * A listener that asserts certain events are logged by the given EventLoggingListener. + * This is necessary because events are posted asynchronously in a different thread. + */ + private class EventExistenceListener(eventLogger: EventLoggingListener) extends SparkListener { + var jobStarted = false + var jobEnded = false + var appEnded = false + + override def onJobStart(jobStart: SparkListenerJobStart) { + assertEventsExist(eventLogger, Seq[String]( + Utils.getFormattedClassName(SparkListenerApplicationStart), + Utils.getFormattedClassName(SparkListenerBlockManagerAdded), + Utils.getFormattedClassName(SparkListenerEnvironmentUpdate) + )) + jobStarted = true + } + + override def onJobEnd(jobEnd: SparkListenerJobEnd) { + assertEventsExist(eventLogger, Seq[String]( + Utils.getFormattedClassName(SparkListenerJobStart), + Utils.getFormattedClassName(SparkListenerJobEnd), + Utils.getFormattedClassName(SparkListenerStageSubmitted), + Utils.getFormattedClassName(SparkListenerStageCompleted), + Utils.getFormattedClassName(SparkListenerTaskStart), + Utils.getFormattedClassName(SparkListenerTaskEnd) + )) + jobEnded = true + } + + override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { + assertEventsExist(eventLogger, Seq[String]( + Utils.getFormattedClassName(SparkListenerApplicationEnd) + )) + appEnded = true + } + + def assertAllCallbacksInvoked() { + assert(jobStarted, "JobStart callback not invoked!") + assert(jobEnded, "JobEnd callback not invoked!") + assert(appEnded, "ApplicationEnd callback not invoked!") + } + } + + + /* -------------------------------------------------------- * + * Helper methods for validating state of the special files * + * -------------------------------------------------------- */ + + private def eventLogsExist(logFiles: Array[FileStatus]): Boolean = { + logFiles.map(_.getPath.getName).exists(EventLoggingListener.isEventLogFile) + } + + private def sparkVersionExists(logFiles: Array[FileStatus]): Boolean = { + logFiles.map(_.getPath.getName).exists(EventLoggingListener.isSparkVersionFile) + } + + private def compressionCodecExists(logFiles: Array[FileStatus]): Boolean = { + logFiles.map(_.getPath.getName).exists(EventLoggingListener.isCompressionCodecFile) + } + + private def applicationCompleteExists(logFiles: Array[FileStatus]): Boolean = { + logFiles.map(_.getPath.getName).exists(EventLoggingListener.isApplicationCompleteFile) + } + + private def assertSparkVersionIsValid(logFiles: Array[FileStatus]) { + val file = logFiles.map(_.getPath.getName).find(EventLoggingListener.isSparkVersionFile) + assert(file.isDefined) + assert(EventLoggingListener.parseSparkVersion(file.get) === SparkContext.SPARK_VERSION) + } + + private def assertCompressionCodecIsValid(logFiles: Array[FileStatus], compressionCodec: String) { + val file = logFiles.map(_.getPath.getName).find(EventLoggingListener.isCompressionCodecFile) + assert(file.isDefined) + assert(EventLoggingListener.parseCompressionCodec(file.get) === compressionCodec) + } + +} + + +object EventLoggingListenerSuite { + + /** Get a SparkConf with event logging enabled. */ + def getLoggingConf(logDir: Path, compressionCodec: Option[String] = None) = { + val conf = new SparkConf + conf.set("spark.eventLog.enabled", "true") + conf.set("spark.eventLog.testing", "true") + conf.set("spark.eventLog.dir", logDir.toString) + compressionCodec.foreach { codec => + conf.set("spark.eventLog.compress", "true") + conf.set("spark.io.compression.codec", codec) + } + conf + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala new file mode 100644 index 0000000000000..d81499ac6abef --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -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. + */ + +package org.apache.spark.scheduler + +import java.io.{File, PrintWriter} + +import com.google.common.io.Files +import org.json4s.jackson.JsonMethods._ +import org.scalatest.{BeforeAndAfter, FunSuite} + +import org.apache.spark.SparkContext._ +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.{JsonProtocol, Utils} + +/** + * Test whether ReplayListenerBus replays events from logs correctly. + */ +class ReplayListenerSuite extends FunSuite with BeforeAndAfter { + private val fileSystem = Utils.getHadoopFileSystem("/") + private val allCompressionCodecs = Seq[String]( + "org.apache.spark.io.LZFCompressionCodec", + "org.apache.spark.io.SnappyCompressionCodec" + ) + private var testDir: File = _ + + before { + testDir = Files.createTempDir() + testDir.deleteOnExit() + } + + after { + Utils.deleteRecursively(testDir) + } + + test("Simple replay") { + testSimpleReplay() + } + + test("Simple replay with compression") { + allCompressionCodecs.foreach { codec => + testSimpleReplay(Some(codec)) + } + } + + // This assumes the correctness of EventLoggingListener + test("End-to-end replay") { + testApplicationReplay() + } + + // This assumes the correctness of EventLoggingListener + test("End-to-end replay with compression") { + allCompressionCodecs.foreach { codec => + testApplicationReplay(Some(codec)) + } + } + + + /* ----------------- * + * Actual test logic * + * ----------------- */ + + /** + * Test simple replaying of events. + */ + private def testSimpleReplay(codecName: Option[String] = None) { + val logFilePath = Utils.getFilePath(testDir, "events.txt") + val codec = codecName.map(getCompressionCodec) + val fstream = fileSystem.create(logFilePath) + val cstream = codec.map(_.compressedOutputStream(fstream)).getOrElse(fstream) + val writer = new PrintWriter(cstream) + val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", 125L, "Mickey") + val applicationEnd = SparkListenerApplicationEnd(1000L) + writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) + writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) + writer.close() + val replayer = new ReplayListenerBus(Seq(logFilePath), fileSystem, codec) + val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath, codecName) + val eventMonster = new EventMonster(conf) + replayer.addListener(eventMonster) + replayer.replay() + assert(eventMonster.loggedEvents.size === 2) + assert(eventMonster.loggedEvents(0) === JsonProtocol.sparkEventToJson(applicationStart)) + assert(eventMonster.loggedEvents(1) === JsonProtocol.sparkEventToJson(applicationEnd)) + } + + /** + * Test end-to-end replaying of events. + * + * This test runs a few simple jobs with event logging enabled, and compares each emitted + * event to the corresponding event replayed from the event logs. This test makes the + * assumption that the event logging behavior is correct (tested in a separate suite). + */ + private def testApplicationReplay(codecName: Option[String] = None) { + val logDirPath = Utils.getFilePath(testDir, "test-replay") + val conf = EventLoggingListenerSuite.getLoggingConf(logDirPath, codecName) + val sc = new SparkContext("local-cluster[2,1,512]", "Test replay", conf) + + // Run a few jobs + sc.parallelize(1 to 100, 1).count() + sc.parallelize(1 to 100, 2).map(i => (i, i)).count() + sc.parallelize(1 to 100, 3).map(i => (i, i)).groupByKey().count() + sc.parallelize(1 to 100, 4).map(i => (i, i)).groupByKey().persist().count() + sc.stop() + + // Prepare information needed for replay + val codec = codecName.map(getCompressionCodec) + val applications = fileSystem.listStatus(logDirPath) + assert(applications != null && applications.size > 0) + val eventLogDir = applications.sortBy(_.getAccessTime).last + assert(eventLogDir.isDir) + val logFiles = fileSystem.listStatus(eventLogDir.getPath) + assert(logFiles != null && logFiles.size > 0) + val logFile = logFiles.find(_.getPath.getName.startsWith("EVENT_LOG_")) + assert(logFile.isDefined) + val logFilePath = logFile.get.getPath + + // Replay events + val replayer = new ReplayListenerBus(Seq(logFilePath), fileSystem, codec) + val eventMonster = new EventMonster(conf) + replayer.addListener(eventMonster) + replayer.replay() + + // Verify the same events are replayed in the same order + assert(sc.eventLogger.isDefined) + val originalEvents = sc.eventLogger.get.loggedEvents + val replayedEvents = eventMonster.loggedEvents + originalEvents.zip(replayedEvents).foreach { case (e1, e2) => assert(e1 === e2) } + } + + /** + * A simple listener that buffers all the events it receives. + * + * The event buffering functionality must be implemented within EventLoggingListener itself. + * This is because of the following race condition: the event may be mutated between being + * processed by one listener and being processed by another. Thus, in order to establish + * a fair comparison between the original events and the replayed events, both functionalities + * must be implemented within one listener (i.e. the EventLoggingListener). + * + * This child listener inherits only the event buffering functionality, but does not actually + * log the events. + */ + private class EventMonster(conf: SparkConf) extends EventLoggingListener("test", conf) { + logger.close() + } + + private def getCompressionCodec(codecName: String) = { + val conf = new SparkConf + conf.set("spark.io.compression.codec", codecName) + CompressionCodec.createCodec(conf) + } + +} 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 7c843772bc2e0..abd7b22310f1a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -17,17 +17,20 @@ package org.apache.spark.scheduler +import java.util.concurrent.Semaphore + import scala.collection.mutable import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} -import org.scalatest.matchers.ShouldMatchers +import org.scalatest.Matchers import org.apache.spark.{LocalSparkContext, SparkContext} import org.apache.spark.SparkContext._ import org.apache.spark.executor.TaskMetrics -class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatchers - with BeforeAndAfter with BeforeAndAfterAll { +class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers + with BeforeAndAfter with BeforeAndAfterAll { + /** Length of time to wait while draining listener events. */ val WAIT_TIMEOUT_MILLIS = 10000 @@ -35,7 +38,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc sc = new SparkContext("local", "SparkListenerSuite") } - override def afterAll { + override def afterAll() { System.clearProperty("spark.akka.frameSize") } @@ -48,9 +51,9 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, JobSucceeded)) } assert(counter.count === 0) - // Starting listener bus should flush all buffered events (asynchronously, hence the sleep) + // Starting listener bus should flush all buffered events bus.start() - Thread.sleep(1000) + assert(bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(counter.count === 5) // After listener bus has stopped, posting events should not increment counter @@ -72,26 +75,117 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc } } + test("bus.stop() waits for the event queue to completely drain") { + @volatile var drained = false + + // When Listener has started + val listenerStarted = new Semaphore(0) + + // Tells the listener to stop blocking + val listenerWait = new Semaphore(0) + + // When stopper has started + val stopperStarted = new Semaphore(0) + + // When stopper has returned + val stopperReturned = new Semaphore(0) + + class BlockingListener extends SparkListener { + override def onJobEnd(jobEnd: SparkListenerJobEnd) = { + listenerStarted.release() + listenerWait.acquire() + drained = true + } + } + + val bus = new LiveListenerBus + val blockingListener = new BlockingListener + + bus.addListener(blockingListener) + bus.start() + bus.post(SparkListenerJobEnd(0, JobSucceeded)) + + listenerStarted.acquire() + // Listener should be blocked after start + assert(!drained) + + new Thread("ListenerBusStopper") { + override def run() { + stopperStarted.release() + // stop() will block until notify() is called below + bus.stop() + stopperReturned.release() + } + }.start() + + stopperStarted.acquire() + // Listener should remain blocked after stopper started + assert(!drained) + + // unblock Listener to let queue drain + listenerWait.release() + stopperReturned.acquire() + assert(drained) + } + test("basic creation of StageInfo") { val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) val rdd1 = sc.parallelize(1 to 100, 4) val rdd2 = rdd1.map(_.toString) rdd2.setName("Target RDD") - rdd2.count + rdd2.count() assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be {1} val (stageInfo, taskInfoMetrics) = listener.stageInfos.head - stageInfo.rddInfo.name should be {"Target RDD"} + stageInfo.rddInfos.size should be {2} + stageInfo.rddInfos.forall(_.numPartitions == 4) should be {true} + stageInfo.rddInfos.exists(_.name == "Target RDD") should be {true} stageInfo.numTasks should be {4} - stageInfo.rddInfo.numPartitions should be {4} stageInfo.submissionTime should be ('defined) stageInfo.completionTime should be ('defined) taskInfoMetrics.length should be {4} } + test("basic creation of StageInfo with shuffle") { + val listener = new SaveStageAndTaskInfo + sc.addSparkListener(listener) + val rdd1 = sc.parallelize(1 to 100, 4) + val rdd2 = rdd1.filter(_ % 2 == 0).map(i => (i, i)) + val rdd3 = rdd2.reduceByKey(_ + _) + rdd1.setName("Un") + rdd2.setName("Deux") + rdd3.setName("Trois") + + rdd1.count() + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + listener.stageInfos.size should be {1} + val stageInfo1 = listener.stageInfos.keys.find(_.stageId == 0).get + stageInfo1.rddInfos.size should be {1} // ParallelCollectionRDD + stageInfo1.rddInfos.forall(_.numPartitions == 4) should be {true} + stageInfo1.rddInfos.exists(_.name == "Un") should be {true} + listener.stageInfos.clear() + + rdd2.count() + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + listener.stageInfos.size should be {1} + val stageInfo2 = listener.stageInfos.keys.find(_.stageId == 1).get + stageInfo2.rddInfos.size should be {3} // ParallelCollectionRDD, FilteredRDD, MappedRDD + stageInfo2.rddInfos.forall(_.numPartitions == 4) should be {true} + stageInfo2.rddInfos.exists(_.name == "Deux") should be {true} + listener.stageInfos.clear() + + rdd3.count() + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + listener.stageInfos.size should be {2} // Shuffle map stage + result stage + val stageInfo3 = listener.stageInfos.keys.find(_.stageId == 2).get + stageInfo3.rddInfos.size should be {2} // ShuffledRDD, MapPartitionsRDD + stageInfo3.rddInfos.forall(_.numPartitions == 4) should be {true} + stageInfo3.rddInfos.exists(_.name == "Trois") should be {true} + } + test("StageInfo with fewer tasks than partitions") { val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) @@ -104,7 +198,8 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc listener.stageInfos.size should be {1} val (stageInfo, _) = listener.stageInfos.head stageInfo.numTasks should be {2} - stageInfo.rddInfo.numPartitions should be {4} + stageInfo.rddInfos.size should be {2} + stageInfo.rddInfos.forall(_.numPartitions == 4) should be {true} } test("local metrics") { @@ -112,7 +207,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc sc.addSparkListener(listener) sc.addSparkListener(new StatsReportListener) // just to make sure some of the tasks take a noticeable amount of time - val w = {i:Int => + val w = { i: Int => if (i == 0) Thread.sleep(100) i @@ -144,19 +239,22 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc checkNonZeroAvg( taskInfoMetrics.map(_._2.executorDeserializeTime), stageInfo + " executorDeserializeTime") - if (stageInfo.rddInfo.name == d4.name) { + + /* Test is disabled (SEE SPARK-2208) + if (stageInfo.rddInfos.exists(_.name == d4.name)) { checkNonZeroAvg( taskInfoMetrics.map(_._2.shuffleReadMetrics.get.fetchWaitTime), stageInfo + " fetchWaitTime") } + */ taskInfoMetrics.foreach { case (taskInfo, taskMetrics) => taskMetrics.resultSize should be > (0l) - if (stageInfo.rddInfo.name == d2.name || stageInfo.rddInfo.name == d3.name) { + if (stageInfo.rddInfos.exists(info => info.name == d2.name || info.name == d3.name)) { taskMetrics.shuffleWriteMetrics should be ('defined) taskMetrics.shuffleWriteMetrics.get.shuffleBytesWritten should be > (0l) } - if (stageInfo.rddInfo.name == d4.name) { + if (stageInfo.rddInfos.exists(_.name == d4.name)) { taskMetrics.shuffleReadMetrics should be ('defined) val sm = taskMetrics.shuffleReadMetrics.get sm.totalBlocksFetched should be > (0) @@ -171,7 +269,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc test("onTaskGettingResult() called when result fetched remotely") { val listener = new SaveTaskEvents sc.addSparkListener(listener) - + // Make a task whose result is larger than the akka frame size System.setProperty("spark.akka.frameSize", "1") val akkaFrameSize = @@ -191,7 +289,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc test("onTaskGettingResult() not called when result sent directly") { val listener = new SaveTaskEvents sc.addSparkListener(listener) - + // Make a task whose result is larger than the akka frame size val result = sc.parallelize(Seq(1), 1).map(2 * _).reduce { case (x, y) => x } assert(result === 2) @@ -236,16 +334,47 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc } } - def checkNonZeroAvg(m: Traversable[Long], msg: String) { + test("SparkListener moves on if a listener throws an exception") { + val badListener = new BadListener + val jobCounter1 = new BasicJobCounter + val jobCounter2 = new BasicJobCounter + val bus = new LiveListenerBus + + // Propagate events to bad listener first + bus.addListener(badListener) + bus.addListener(jobCounter1) + bus.addListener(jobCounter2) + bus.start() + + // Post events to all listeners, and wait until the queue is drained + (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, JobSucceeded)) } + assert(bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + + // The exception should be caught, and the event should be propagated to other listeners + assert(bus.listenerThreadIsAlive) + assert(jobCounter1.count === 5) + assert(jobCounter2.count === 5) + } + + /** + * Assert that the given list of numbers has an average that is greater than zero. + */ + private def checkNonZeroAvg(m: Traversable[Long], msg: String) { assert(m.sum / m.size.toDouble > 0.0, msg) } - class BasicJobCounter extends SparkListener { + /** + * A simple listener that counts the number of jobs observed. + */ + private class BasicJobCounter extends SparkListener { var count = 0 override def onJobEnd(job: SparkListenerJobEnd) = count += 1 } - class SaveStageAndTaskInfo extends SparkListener { + /** + * A simple listener that saves all task infos and task metrics. + */ + private class SaveStageAndTaskInfo extends SparkListener { val stageInfos = mutable.Map[StageInfo, Seq[(TaskInfo, TaskMetrics)]]() var taskInfoMetrics = mutable.Buffer[(TaskInfo, TaskMetrics)]() @@ -263,7 +392,10 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc } } - class SaveTaskEvents extends SparkListener { + /** + * A simple listener that saves the task indices for all task events. + */ + private class SaveTaskEvents extends SparkListener { val startedTasks = new mutable.HashSet[Int]() val startedGettingResultTasks = new mutable.HashSet[Int]() val endedTasks = new mutable.HashSet[Int]() @@ -282,4 +414,12 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc startedGettingResultTasks += taskGettingResult.taskInfo.index } } + + /** + * A simple listener that throws an exception on job end. + */ + private class BadListener extends SparkListener { + override def onJobEnd(jobEnd: SparkListenerJobEnd) = { throw new Exception } + } + } 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 356e28dd19bc5..7532da88c6065 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -117,7 +117,7 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin } def resourceOffer(rootPool: Pool): Int = { - val taskSetQueue = rootPool.getSortedTaskSetQueue() + val taskSetQueue = rootPool.getSortedTaskSetQueue /* Just for Test*/ for (manager <- taskSetQueue) { logInfo("parentName:%s, parent running tasks:%d, name:%s,runningTasks:%d".format( @@ -264,7 +264,7 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin test("Scheduler does not always schedule tasks on the same workers") { sc = new SparkContext("local", "TaskSchedulerImplSuite") - val taskScheduler = new TaskSchedulerImpl(sc) + val taskScheduler = new TaskSchedulerImpl(sc) taskScheduler.initialize(new FakeSchedulerBackend) // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. val dagScheduler = new DAGScheduler(sc, taskScheduler) { @@ -305,7 +305,7 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} override def executorAdded(execId: String, host: String) {} } - + taskScheduler.setDAGScheduler(dagScheduler) // Give zero core offers. Should not generate any tasks val zeroCoreWorkerOffers = Seq(new WorkerOffer("executor0", "host0", 0), new WorkerOffer("executor1", "host1", 0)) 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 c92b6dc96c8eb..6f1fd25764544 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -141,6 +141,22 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(sched.finishedManagers.contains(manager)) } + test("skip unsatisfiable locality levels") { + sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execC", "host2")) + val taskSet = FakeTask.createTaskSet(1, Seq(TaskLocation("host1", "execB"))) + val clock = new FakeClock + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + + // An executor that is not NODE_LOCAL should be rejected. + assert(manager.resourceOffer("execC", "host2", ANY) === None) + + // Because there are no alive PROCESS_LOCAL executors, the base locality level should be + // NODE_LOCAL. So, we should schedule the task on this offered NODE_LOCAL executor before + // any of the locality wait timers expire. + assert(manager.resourceOffer("execA", "host1", ANY).get.index === 0) + } + test("basic delay scheduling") { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) 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 f28bc7b47787a..5d10f10ad1f63 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.serializer import scala.collection.mutable +import scala.reflect.ClassTag import com.esotericsoftware.kryo.Kryo import org.scalatest.FunSuite @@ -31,7 +32,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("basic types") { val ser = new KryoSerializer(conf).newInstance() - def check[T](t: T) { + def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } check(1) @@ -61,7 +62,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("pairs") { val ser = new KryoSerializer(conf).newInstance() - def check[T](t: T) { + def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } check((1, 1)) @@ -85,7 +86,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("Scala data structures") { val ser = new KryoSerializer(conf).newInstance() - def check[T](t: T) { + def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } check(List[Int]()) @@ -108,7 +109,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("ranges") { val ser = new KryoSerializer(conf).newInstance() - def check[T](t: T) { + def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) // Check that very long ranges don't get written one element at a time assert(ser.serialize(t).limit < 100) @@ -129,7 +130,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("custom registrator") { val ser = new KryoSerializer(conf).newInstance() - def check[T](t: T) { + def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } 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 b6dd0526105a0..d7dbe5164b7f6 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -17,23 +17,30 @@ package org.apache.spark.storage -import java.nio.ByteBuffer +import java.nio.{ByteBuffer, MappedByteBuffer} +import java.util.Arrays import akka.actor._ -import org.scalatest.BeforeAndAfter -import org.scalatest.FunSuite -import org.scalatest.PrivateMethodTester +import org.apache.spark.SparkConf +import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} +import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} +import org.mockito.Mockito.{mock, when} +import org.scalatest.{BeforeAndAfter, FunSuite, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ -import org.scalatest.matchers.ShouldMatchers._ +import org.scalatest.Matchers import org.scalatest.time.SpanSugar._ -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} -class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester { +import scala.language.implicitConversions +import scala.language.postfixOps + +class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter + with PrivateMethodTester { private val conf = new SparkConf(false) var store: BlockManager = null var store2: BlockManager = null @@ -42,6 +49,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT var oldArch: String = null conf.set("spark.authenticate", "false") val securityMgr = new SecurityManager(conf) + val mapOutputTracker = new MapOutputTrackerMaster(conf) // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test conf.set("spark.kryoserializer.buffer.mb", "1") @@ -71,8 +79,6 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } after { - System.clearProperty("spark.driver.port") - if (store != null) { store.stop() store = null @@ -130,7 +136,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("master + 1 manager interaction") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -160,9 +167,10 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("master + 2 managers interaction") { - store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer(conf), 2000, conf, - securityMgr) + securityMgr, mapOutputTracker) val peers = master.getPeers(store.blockManagerId, 1) assert(peers.size === 1, "master did not return the other manager as a peer") @@ -177,7 +185,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("removing block") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -225,7 +234,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("removing rdd") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -257,9 +267,82 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT master.getLocations(rdd(0, 1)) should have size 0 } + test("removing broadcast") { + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) + val driverStore = store + val executorStore = new BlockManager("executor", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) + val a1 = new Array[Byte](400) + val a2 = new Array[Byte](400) + val a3 = new Array[Byte](400) + val a4 = new Array[Byte](400) + + val broadcast0BlockId = BroadcastBlockId(0) + val broadcast1BlockId = BroadcastBlockId(1) + val broadcast2BlockId = BroadcastBlockId(2) + val broadcast2BlockId2 = BroadcastBlockId(2, "_") + + // insert broadcast blocks in both the stores + Seq(driverStore, executorStore).foreach { case s => + s.putSingle(broadcast0BlockId, a1, StorageLevel.DISK_ONLY) + s.putSingle(broadcast1BlockId, a2, StorageLevel.DISK_ONLY) + s.putSingle(broadcast2BlockId, a3, StorageLevel.DISK_ONLY) + s.putSingle(broadcast2BlockId2, a4, StorageLevel.DISK_ONLY) + } + + // verify whether the blocks exist in both the stores + Seq(driverStore, executorStore).foreach { case s => + s.getLocal(broadcast0BlockId) should not be (None) + s.getLocal(broadcast1BlockId) should not be (None) + s.getLocal(broadcast2BlockId) should not be (None) + s.getLocal(broadcast2BlockId2) should not be (None) + } + + // remove broadcast 0 block only from executors + master.removeBroadcast(0, removeFromMaster = false, blocking = true) + + // only broadcast 0 block should be removed from the executor store + executorStore.getLocal(broadcast0BlockId) should be (None) + executorStore.getLocal(broadcast1BlockId) should not be (None) + executorStore.getLocal(broadcast2BlockId) should not be (None) + + // nothing should be removed from the driver store + driverStore.getLocal(broadcast0BlockId) should not be (None) + driverStore.getLocal(broadcast1BlockId) should not be (None) + driverStore.getLocal(broadcast2BlockId) should not be (None) + + // remove broadcast 0 block from the driver as well + master.removeBroadcast(0, removeFromMaster = true, blocking = true) + driverStore.getLocal(broadcast0BlockId) should be (None) + driverStore.getLocal(broadcast1BlockId) should not be (None) + + // remove broadcast 1 block from both the stores asynchronously + // and verify all broadcast 1 blocks have been removed + master.removeBroadcast(1, removeFromMaster = true, blocking = false) + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + driverStore.getLocal(broadcast1BlockId) should be (None) + executorStore.getLocal(broadcast1BlockId) should be (None) + } + + // remove broadcast 2 from both the stores asynchronously + // and verify all broadcast 2 blocks have been removed + master.removeBroadcast(2, removeFromMaster = true, blocking = false) + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + driverStore.getLocal(broadcast2BlockId) should be (None) + driverStore.getLocal(broadcast2BlockId2) should be (None) + executorStore.getLocal(broadcast2BlockId) should be (None) + executorStore.getLocal(broadcast2BlockId2) should be (None) + } + executorStore.stop() + driverStore.stop() + store = null + } + test("reregistration on heart beat") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) @@ -275,7 +358,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("reregistration on block update") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) @@ -294,7 +378,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("reregistration doesn't dead lock") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = List(new Array[Byte](400)) @@ -331,7 +416,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -350,7 +436,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU storage with serialization") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -369,7 +456,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU for partitions of same RDD") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -388,7 +476,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU for partitions of multiple RDDs") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) store.putSingle(rdd(0, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 2), new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(1, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) @@ -414,7 +503,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT // 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) if (tachyonUnitTestEnabled) { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -430,7 +520,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("on-disk storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -443,7 +534,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -458,7 +550,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with getLocalBytes") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -473,7 +566,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with serialization") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -488,7 +582,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with serialization and getLocalBytes") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -503,7 +598,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("LRU with mixed storage levels") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -525,7 +621,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU with streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) @@ -549,7 +646,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("LRU with mixed storage levels and streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) @@ -595,7 +693,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("overly large block") { - store = new BlockManager("", actorSystem, master, serializer, 500, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 500, conf, + securityMgr, mapOutputTracker) store.putSingle("a1", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) assert(store.getSingle("a1") === None, "a1 was in store") store.putSingle("a2", new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK) @@ -606,7 +705,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("block compression") { try { conf.set("spark.shuffle.compress", "true") - store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100, "shuffle_0_0_0 was not compressed") @@ -614,7 +714,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store = null conf.set("spark.shuffle.compress", "false") - store = new BlockManager("exec2", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec2", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 1000, "shuffle_0_0_0 was compressed") @@ -622,7 +723,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store = null conf.set("spark.broadcast.compress", "true") - store = new BlockManager("exec3", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec3", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 100, "broadcast_0 was not compressed") @@ -630,28 +732,32 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store = null conf.set("spark.broadcast.compress", "false") - store = new BlockManager("exec4", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec4", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 1000, "broadcast_0 was compressed") store.stop() store = null conf.set("spark.rdd.compress", "true") - store = new BlockManager("exec5", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec5", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) <= 100, "rdd_0_0 was not compressed") store.stop() store = null conf.set("spark.rdd.compress", "false") - store = new BlockManager("exec6", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec6", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) >= 1000, "rdd_0_0 was compressed") store.stop() store = null // Check that any other block types are also kept uncompressed - store = new BlockManager("exec7", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec7", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle("other_block", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) assert(store.memoryStore.getSize("other_block") >= 1000, "other_block was compressed") store.stop() @@ -666,7 +772,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("block store put failure") { // Use Java serializer so we can create an unserializable error. store = new BlockManager("", actorSystem, master, new JavaSerializer(conf), 1200, conf, - securityMgr) + securityMgr, mapOutputTracker) // The put should fail since a1 is not serializable. class UnserializableClass @@ -681,8 +787,56 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } } + test("reads of memory-mapped and non memory-mapped files are equivalent") { + val confKey = "spark.storage.memoryMapThreshold" + + // Create a non-trivial (not all zeros) byte array + var counter = 0.toByte + def incr = {counter = (counter + 1).toByte; counter;} + val bytes = Array.fill[Byte](1000)(incr) + val byteBuffer = ByteBuffer.wrap(bytes) + + val blockId = BlockId("rdd_1_2") + + // This sequence of mocks makes these tests fairly brittle. It would + // be nice to refactor classes involved in disk storage in a way that + // allows for easier testing. + val blockManager = mock(classOf[BlockManager]) + val shuffleBlockManager = mock(classOf[ShuffleBlockManager]) + when(shuffleBlockManager.conf).thenReturn(conf) + val diskBlockManager = new DiskBlockManager(shuffleBlockManager, + System.getProperty("java.io.tmpdir")) + + when(blockManager.conf).thenReturn(conf.clone.set(confKey, 0.toString)) + 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)) + val diskStoreNotMapped = new DiskStore(blockManager, diskBlockManager) + diskStoreNotMapped.putBytes(blockId, byteBuffer, StorageLevel.DISK_ONLY) + val notMapped = diskStoreNotMapped.getBytes(blockId).get + + // Not possible to do isInstanceOf due to visibility of HeapByteBuffer + assert(notMapped.getClass.getName.endsWith("HeapByteBuffer"), + "Expected HeapByteBuffer for un-mapped read") + assert(mapped.isInstanceOf[MappedByteBuffer], "Expected MappedByteBuffer for mapped read") + + def arrayFromByteBuffer(in: ByteBuffer): Array[Byte] = { + val array = new Array[Byte](in.remaining()) + in.get(array) + array + } + + val mappedAsArray = arrayFromByteBuffer(mapped) + val notMappedAsArray = arrayFromByteBuffer(notMapped) + assert(Arrays.equals(mappedAsArray, bytes)) + assert(Arrays.equals(notMappedAsArray, bytes)) + } + test("updated block statuses") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val list = List.fill(2)(new Array[Byte](200)) val bigList = List.fill(8)(new Array[Byte](200)) @@ -735,8 +889,83 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(!store.get("list5").isDefined, "list5 was in store") } + test("query block statuses") { + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) + val list = List.fill(2)(new Array[Byte](200)) + + // Tell master. By LRU, only list2 and list3 remains. + store.put("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.put("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + + // getLocations and getBlockStatus should yield the same locations + assert(store.master.getLocations("list1").size === 0) + assert(store.master.getLocations("list2").size === 1) + assert(store.master.getLocations("list3").size === 1) + assert(store.master.getBlockStatus("list1", askSlaves = false).size === 0) + assert(store.master.getBlockStatus("list2", askSlaves = false).size === 1) + assert(store.master.getBlockStatus("list3", askSlaves = false).size === 1) + assert(store.master.getBlockStatus("list1", askSlaves = true).size === 0) + assert(store.master.getBlockStatus("list2", askSlaves = true).size === 1) + assert(store.master.getBlockStatus("list3", askSlaves = true).size === 1) + + // This time don't tell master and see what happens. By LRU, only list5 and list6 remains. + store.put("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) + store.put("list5", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.put("list6", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) + + // getLocations should return nothing because the master is not informed + // getBlockStatus without asking slaves should have the same result + // getBlockStatus with asking slaves, however, should return the actual block statuses + assert(store.master.getLocations("list4").size === 0) + assert(store.master.getLocations("list5").size === 0) + assert(store.master.getLocations("list6").size === 0) + assert(store.master.getBlockStatus("list4", askSlaves = false).size === 0) + assert(store.master.getBlockStatus("list5", askSlaves = false).size === 0) + assert(store.master.getBlockStatus("list6", askSlaves = false).size === 0) + assert(store.master.getBlockStatus("list4", askSlaves = true).size === 0) + assert(store.master.getBlockStatus("list5", askSlaves = true).size === 1) + assert(store.master.getBlockStatus("list6", askSlaves = true).size === 1) + } + + test("get matching blocks") { + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) + val list = List.fill(2)(new Array[Byte](10)) + + // insert some blocks + store.put("list1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.put("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) + + // insert some more blocks + store.put("newlist1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.put("newlist2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.put("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) + + val blockIds = Seq(RDDBlockId(1, 0), RDDBlockId(1, 1), RDDBlockId(2, 0)) + blockIds.foreach { blockId => + store.put(blockId, list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + } + val matchedBlockIds = store.master.getMatchingBlockIds(_ match { + case RDDBlockId(1, _) => true + case _ => false + }, askSlaves = true) + assert(matchedBlockIds.toSet === Set(RDDBlockId(1, 0), RDDBlockId(1, 1))) + } + test("SPARK-1194 regression: fix the same-RDD rule for cache replacement") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) store.putSingle(rdd(0, 0), new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(1, 0), new Array[Byte](400), StorageLevel.MEMORY_ONLY) // Access rdd_1_0 to ensure it's not least recently used. diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 62f9b3cc7b2c1..aaa7714049732 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -20,20 +20,19 @@ package org.apache.spark.storage import java.io.{File, FileWriter} import scala.collection.mutable +import scala.language.reflectiveCalls import com.google.common.io.Files -import org.scalatest.{BeforeAndAfterEach, FunSuite} +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.apache.spark.SparkConf +import org.apache.spark.util.Utils -class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { +class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll { private val testConf = new SparkConf(false) - val rootDir0 = Files.createTempDir() - rootDir0.deleteOnExit() - val rootDir1 = Files.createTempDir() - rootDir1.deleteOnExit() - val rootDirs = rootDir0.getName + "," + rootDir1.getName - println("Created root dirs: " + rootDirs) + private var rootDir0: File = _ + private var rootDir1: File = _ + private var rootDirs: String = _ // This suite focuses primarily on consolidation features, // so we coerce consolidation if not already enabled. @@ -47,11 +46,32 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { var diskBlockManager: DiskBlockManager = _ + override def beforeAll() { + super.beforeAll() + rootDir0 = Files.createTempDir() + rootDir0.deleteOnExit() + rootDir1 = Files.createTempDir() + rootDir1.deleteOnExit() + rootDirs = rootDir0.getAbsolutePath + "," + rootDir1.getAbsolutePath + println("Created root dirs: " + rootDirs) + } + + override def afterAll() { + super.afterAll() + Utils.deleteRecursively(rootDir0) + Utils.deleteRecursively(rootDir1) + } + override def beforeEach() { diskBlockManager = new DiskBlockManager(shuffleBlockManager, rootDirs) shuffleBlockManager.idToSegmentMap.clear() } + override def afterEach() { + diskBlockManager.stop() + shuffleBlockManager.idToSegmentMap.clear() + } + test("basic block creation") { val blockId = new TestBlockId("test") assertSegmentEquals(blockId, blockId.name, 0, 0) @@ -59,8 +79,16 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { val newFile = diskBlockManager.getFile(blockId) writeToFile(newFile, 10) assertSegmentEquals(blockId, blockId.name, 0, 10) - + assert(diskBlockManager.containsBlock(blockId)) newFile.delete() + assert(!diskBlockManager.containsBlock(blockId)) + } + + test("enumerating blocks") { + val ids = (1 to 100).map(i => TestBlockId("test_" + i)) + val files = ids.map(id => diskBlockManager.getFile(id)) + files.foreach(file => writeToFile(file, 10)) + assert(diskBlockManager.getAllBlocks.toSet === ids.toSet) } test("block appending") { diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 45c322427930d..038746d2eda4b 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -18,23 +18,89 @@ package org.apache.spark.ui import java.net.ServerSocket +import javax.servlet.http.HttpServletRequest +import scala.io.Source +import scala.language.postfixOps import scala.util.{Failure, Success, Try} import org.eclipse.jetty.server.Server import org.eclipse.jetty.servlet.ServletContextHandler import org.scalatest.FunSuite +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ -import org.apache.spark.SparkConf +import org.apache.spark.{SparkContext, SparkConf} +import org.apache.spark.LocalSparkContext._ +import scala.xml.Node class UISuite extends FunSuite { - test("jetty port increases under contention") { + + ignore("basic ui visibility") { + withSpark(new SparkContext("local", "test")) { sc => + // test if the ui is visible, and all the expected tabs are visible + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL(sc.ui.appUIAddress).mkString + assert(!html.contains("random data that should not be present")) + assert(html.toLowerCase.contains("stages")) + assert(html.toLowerCase.contains("storage")) + assert(html.toLowerCase.contains("environment")) + assert(html.toLowerCase.contains("executors")) + } + } + } + + ignore("visibility at localhost:4040") { + withSpark(new SparkContext("local", "test")) { sc => + // test if visible from http://localhost:4040 + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL("http://localhost:4040").mkString + assert(html.toLowerCase.contains("stages")) + } + } + } + + ignore("attaching a new tab") { + withSpark(new SparkContext("local", "test")) { sc => + val sparkUI = sc.ui + + val newTab = new WebUITab(sparkUI, "foo") { + attachPage(new WebUIPage("") { + def render(request: HttpServletRequest): Seq[Node] = { + "html magic" + } + }) + } + sparkUI.attachTab(newTab) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL(sc.ui.appUIAddress).mkString + assert(!html.contains("random data that should not be present")) + + // check whether new page exists + assert(html.toLowerCase.contains("foo")) + + // check whether other pages still exist + assert(html.toLowerCase.contains("stages")) + assert(html.toLowerCase.contains("storage")) + assert(html.toLowerCase.contains("environment")) + assert(html.toLowerCase.contains("executors")) + } + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL(sc.ui.appUIAddress.stripSuffix("/") + "/foo").mkString + // check whether new page exists + assert(html.contains("magic")) + } + } + } + + test("jetty selects different port under contention") { val startPort = 4040 val server = new Server(startPort) Try { server.start() } match { - case Success(s) => - case Failure(e) => + case Success(s) => + case Failure(e) => // Either case server port is busy hence setup for test complete } val serverInfo1 = JettyUtils.startJettyServer( @@ -44,8 +110,9 @@ class UISuite extends FunSuite { // Allow some wiggle room in case ports on the machine are under contention val boundPort1 = serverInfo1.boundPort val boundPort2 = serverInfo2.boundPort - assert(boundPort1 > startPort && boundPort1 < startPort + 10) - assert(boundPort2 > boundPort1 && boundPort2 < boundPort1 + 10) + assert(boundPort1 != startPort) + assert(boundPort2 != startPort) + assert(boundPort1 != boundPort2) } test("jetty binds to port 0 correctly") { @@ -60,4 +127,18 @@ class UISuite extends FunSuite { case Failure(e) => } } + + test("verify appUIAddress contains the scheme") { + withSpark(new SparkContext("local", "test")) { sc => + val uiAddress = sc.ui.appUIAddress + assert(uiAddress.equals("http://" + sc.ui.appUIHostPort)) + } + } + + test("verify appUIAddress contains the port") { + withSpark(new SparkContext("local", "test")) { sc => + val splitUIAddress = sc.ui.appUIAddress.split(':') + assert(splitUIAddress(2).toInt == sc.ui.boundPort) + } + } } 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 beac656f573b4..c3a14f48de38e 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 @@ -18,26 +18,26 @@ package org.apache.spark.ui.jobs import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers +import org.scalatest.Matchers -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, Success} +import org.apache.spark.{LocalSparkContext, SparkConf, Success} import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics} import org.apache.spark.scheduler._ import org.apache.spark.util.Utils -class JobProgressListenerSuite extends FunSuite with LocalSparkContext with ShouldMatchers { +class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matchers { test("test LRU eviction of stages") { val conf = new SparkConf() conf.set("spark.ui.retainedStages", 5.toString) val listener = new JobProgressListener(conf) def createStageStartEvent(stageId: Int) = { - val stageInfo = new StageInfo(stageId, stageId.toString, 0, null) + val stageInfo = new StageInfo(stageId, stageId.toString, 0, null, "") SparkListenerStageSubmitted(stageInfo) } def createStageEndEvent(stageId: Int) = { - val stageInfo = new StageInfo(stageId, stageId.toString, 0, null) + val stageInfo = new StageInfo(stageId, stageId.toString, 0, null, "") SparkListenerStageCompleted(stageInfo) } 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 439e5644e20a3..054ef54e746a5 100644 --- a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.util import org.scalatest.FunSuite import org.apache.spark.LocalSparkContext._ -import org.apache.spark.SparkContext +import org.apache.spark.{SparkContext, SparkException} class ClosureCleanerSuite extends FunSuite { test("closures inside an object") { @@ -50,6 +50,19 @@ 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] { + 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) + } } // A non-serializable class we create in closures to make sure that we aren't @@ -69,7 +82,7 @@ object TestObject { class TestClass extends Serializable { var x = 5 - + def getX = x def run(): Int = { @@ -108,6 +121,30 @@ class TestClassWithoutFieldAccess { } } +object TestObjectWithBogusReturns { + def run(): Int = { + withSpark(new SparkContext("local", "test")) { sc => + val nums = sc.parallelize(Array(1, 2, 3, 4)) + // this return is invalid since it will transfer control outside the closure + nums.map {x => return 1 ; x * 2} + 1 + } + } +} + +object TestObjectWithNestedReturns { + def run(): Int = { + withSpark(new SparkContext("local", "test")) { sc => + val nums = sc.parallelize(Array(1, 2, 3, 4)) + nums.map {x => + // this return is fine since it will not transfer control outside the closure + def foo(): Int = { return 5; 1 } + foo() + } + 1 + } + } +} object TestObjectWithNesting { def run(): Int = { diff --git a/core/src/test/scala/org/apache/spark/util/DistributionSuite.scala b/core/src/test/scala/org/apache/spark/util/DistributionSuite.scala index 63642461e4465..090d48ec921a1 100644 --- a/core/src/test/scala/org/apache/spark/util/DistributionSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/DistributionSuite.scala @@ -18,13 +18,13 @@ package org.apache.spark.util import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers +import org.scalatest.Matchers /** * */ -class DistributionSuite extends FunSuite with ShouldMatchers { +class DistributionSuite extends FunSuite with Matchers { test("summary") { val d = new Distribution((1 to 100).toArray.map{_.toDouble}) val stats = d.statCounter diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala new file mode 100644 index 0000000000000..02e228945bbd9 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -0,0 +1,227 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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._ + +import scala.collection.mutable.HashSet +import scala.reflect._ + +import org.apache.commons.io.{FileUtils, IOUtils} +import org.apache.spark.{Logging, SparkConf} +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.apache.spark.util.logging.{RollingFileAppender, SizeBasedRollingPolicy, TimeBasedRollingPolicy, FileAppender} + +class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { + + val testFile = new File("FileAppenderSuite-test-" + System.currentTimeMillis).getAbsoluteFile + + before { + cleanup() + } + + after { + cleanup() + } + + test("basic file appender") { + val testString = (1 to 1000).mkString(", ") + val inputStream = IOUtils.toInputStream(testString) + val appender = new FileAppender(inputStream, testFile) + inputStream.close() + appender.awaitTermination() + assert(FileUtils.readFileToString(testFile) === testString) + } + + test("rolling file appender - time-based rolling") { + // setup input stream and appender + val testOutputStream = new PipedOutputStream() + val testInputStream = new PipedInputStream(testOutputStream, 100 * 1000) + val rolloverIntervalMillis = 100 + val durationMillis = 1000 + val numRollovers = durationMillis / rolloverIntervalMillis + val textToAppend = (1 to numRollovers).map( _.toString * 10 ) + + val appender = new RollingFileAppender(testInputStream, testFile, + new TimeBasedRollingPolicy(rolloverIntervalMillis, s"--HH-mm-ss-SSSS", false), + new SparkConf(), 10) + + testRolling(appender, testOutputStream, textToAppend, rolloverIntervalMillis) + } + + test("rolling file appender - size-based rolling") { + // setup input stream and appender + val testOutputStream = new PipedOutputStream() + val testInputStream = new PipedInputStream(testOutputStream, 100 * 1000) + val rolloverSize = 1000 + val textToAppend = (1 to 3).map( _.toString * 1000 ) + + val appender = new RollingFileAppender(testInputStream, testFile, + new SizeBasedRollingPolicy(rolloverSize, false), new SparkConf(), 99) + + val files = testRolling(appender, testOutputStream, textToAppend, 0) + files.foreach { file => + logInfo(file.toString + ": " + file.length + " bytes") + assert(file.length <= rolloverSize) + } + } + + test("rolling file appender - cleaning") { + // setup input stream and appender + val testOutputStream = new PipedOutputStream() + val testInputStream = new PipedInputStream(testOutputStream, 100 * 1000) + val conf = new SparkConf().set(RollingFileAppender.RETAINED_FILES_PROPERTY, "10") + val appender = new RollingFileAppender(testInputStream, testFile, + new SizeBasedRollingPolicy(1000, false), conf, 10) + + // send data to appender through the input stream, and wait for the data to be written + val allGeneratedFiles = new HashSet[String]() + val items = (1 to 10).map { _.toString * 10000 } + for (i <- 0 until items.size) { + testOutputStream.write(items(i).getBytes("UTF8")) + testOutputStream.flush() + allGeneratedFiles ++= RollingFileAppender.getSortedRolledOverFiles( + testFile.getParentFile.toString, testFile.getName).map(_.toString) + + Thread.sleep(10) + } + testOutputStream.close() + appender.awaitTermination() + logInfo("Appender closed") + + // 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")) + assert(rolledOverFiles.size > 2) + val earliestRolledOverFile = rolledOverFiles.head + val existingRolledOverFiles = RollingFileAppender.getSortedRolledOverFiles( + testFile.getParentFile.toString, testFile.getName).map(_.toString) + logInfo("Existing rolled over files:\n" + existingRolledOverFiles.mkString("\n")) + assert(!existingRolledOverFiles.toSet.contains(earliestRolledOverFile)) + } + + test("file appender selection") { + // Test whether FileAppender.apply() returns the right type of the FileAppender based + // on SparkConf settings. + + def testAppenderSelection[ExpectedAppender: ClassTag, ExpectedRollingPolicy]( + properties: Seq[(String, String)], expectedRollingPolicyParam: Long = -1): Unit = { + + // Set spark conf properties + val conf = new SparkConf + properties.foreach { p => + conf.set(p._1, p._2) + } + + // Create and test file appender + val testOutputStream = new PipedOutputStream() + val testInputStream = new PipedInputStream(testOutputStream) + val appender = FileAppender(testInputStream, testFile, conf) + assert(appender.isInstanceOf[ExpectedAppender]) + assert(appender.getClass.getSimpleName === + classTag[ExpectedAppender].runtimeClass.getSimpleName) + if (appender.isInstanceOf[RollingFileAppender]) { + val rollingPolicy = appender.asInstanceOf[RollingFileAppender].rollingPolicy + rollingPolicy.isInstanceOf[ExpectedRollingPolicy] + val policyParam = if (rollingPolicy.isInstanceOf[TimeBasedRollingPolicy]) { + rollingPolicy.asInstanceOf[TimeBasedRollingPolicy].rolloverIntervalMillis + } else { + rollingPolicy.asInstanceOf[SizeBasedRollingPolicy].rolloverSizeBytes + } + assert(policyParam === expectedRollingPolicyParam) + } + testOutputStream.close() + appender.awaitTermination() + } + + 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) + + val msInDay = 24 * 60 * 60 * 1000L + val msInHour = 60 * 60 * 1000L + val msInMinute = 60 * 1000L + + // test no strategy -> no rolling + testAppenderSelection[FileAppender, Any](Seq.empty) + + // test time based rolling strategy + testAppenderSelection[RollingFileAppender, Any](rollingStrategy("time"), msInDay) + testAppenderSelection[RollingFileAppender, TimeBasedRollingPolicy]( + rollingStrategy("time") ++ rollingInterval("daily"), msInDay) + testAppenderSelection[RollingFileAppender, TimeBasedRollingPolicy]( + rollingStrategy("time") ++ rollingInterval("hourly"), msInHour) + testAppenderSelection[RollingFileAppender, TimeBasedRollingPolicy]( + rollingStrategy("time") ++ rollingInterval("minutely"), msInMinute) + testAppenderSelection[RollingFileAppender, TimeBasedRollingPolicy]( + rollingStrategy("time") ++ rollingInterval("123456789"), 123456789 * 1000L) + testAppenderSelection[FileAppender, Any]( + rollingStrategy("time") ++ rollingInterval("xyz")) + + // test size based rolling strategy + testAppenderSelection[RollingFileAppender, SizeBasedRollingPolicy]( + rollingStrategy("size") ++ rollingSize("123456789"), 123456789) + testAppenderSelection[FileAppender, Any](rollingSize("xyz")) + + // test illegal strategy + testAppenderSelection[FileAppender, Any](rollingStrategy("xyz")) + } + + /** + * Run the rolling file appender with data and see whether all the data was written correctly + * across rolled over files. + */ + def testRolling( + appender: FileAppender, + outputStream: OutputStream, + textToAppend: Seq[String], + sleepTimeBetweenTexts: Long + ): Seq[File] = { + // send data to appender through the input stream, and wait for the data to be written + val expectedText = textToAppend.mkString("") + for (i <- 0 until textToAppend.size) { + outputStream.write(textToAppend(i).getBytes("UTF8")) + outputStream.flush() + Thread.sleep(sleepTimeBetweenTexts) + } + logInfo("Data sent to appender") + outputStream.close() + appender.awaitTermination() + logInfo("Appender closed") + + // verify whether all the data written to rolled over files is same as expected + val generatedFiles = RollingFileAppender.getSortedRolledOverFiles( + testFile.getParentFile.toString, testFile.getName) + logInfo("Filtered files: \n" + generatedFiles.mkString("\n")) + assert(generatedFiles.size > 1) + val allText = generatedFiles.map { file => + FileUtils.readFileToString(file) + }.mkString("") + assert(allText === expectedText) + generatedFiles + } + + /** Delete all the generated rolledover files */ + def cleanup() { + testFile.getParentFile.listFiles.filter { file => + file.getName.startsWith(testFile.getName) + }.foreach { _.delete() } + } +} diff --git a/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala b/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala new file mode 100644 index 0000000000000..44332fc8dbc23 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala @@ -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. + */ + +package org.apache.spark.util + +import java.io.{File, IOException} + +import scala.io.Source + +import com.google.common.io.Files +import org.apache.hadoop.fs.Path +import org.scalatest.{BeforeAndAfter, FunSuite} + +import org.apache.spark.SparkConf +import org.apache.spark.io.CompressionCodec + +/** + * Test writing files through the FileLogger. + */ +class FileLoggerSuite extends FunSuite with BeforeAndAfter { + private val fileSystem = Utils.getHadoopFileSystem("/") + private val allCompressionCodecs = Seq[String]( + "org.apache.spark.io.LZFCompressionCodec", + "org.apache.spark.io.SnappyCompressionCodec" + ) + private var testDir: File = _ + private var logDirPath: Path = _ + private var logDirPathString: String = _ + + before { + testDir = Files.createTempDir() + logDirPath = Utils.getFilePath(testDir, "test-file-logger") + logDirPathString = logDirPath.toString + } + + after { + Utils.deleteRecursively(testDir) + } + + test("Simple logging") { + testSingleFile() + } + + test ("Simple logging with compression") { + allCompressionCodecs.foreach { codec => + testSingleFile(Some(codec)) + } + } + + test("Logging multiple files") { + testMultipleFiles() + } + + test("Logging multiple files with compression") { + allCompressionCodecs.foreach { codec => + testMultipleFiles(Some(codec)) + } + } + + test("Logging when directory already exists") { + // Create the logging directory multiple times + new FileLogger(logDirPathString, new SparkConf, overwrite = true).start() + new FileLogger(logDirPathString, new SparkConf, overwrite = true).start() + new FileLogger(logDirPathString, new SparkConf, overwrite = true).start() + + // If overwrite is not enabled, an exception should be thrown + intercept[IOException] { + new FileLogger(logDirPathString, new SparkConf, overwrite = false).start() + } + } + + + /* ----------------- * + * Actual test logic * + * ----------------- */ + + /** + * Test logging to a single file. + */ + private def testSingleFile(codecName: Option[String] = None) { + val conf = getLoggingConf(codecName) + val codec = codecName.map { c => CompressionCodec.createCodec(conf) } + val logger = + if (codecName.isDefined) { + new FileLogger(logDirPathString, conf, compress = true) + } else { + new FileLogger(logDirPathString, conf) + } + logger.start() + assert(fileSystem.exists(logDirPath)) + assert(fileSystem.getFileStatus(logDirPath).isDir) + assert(fileSystem.listStatus(logDirPath).size === 0) + + logger.newFile() + val files = fileSystem.listStatus(logDirPath) + assert(files.size === 1) + val firstFile = files.head + val firstFilePath = firstFile.getPath + + logger.log("hello") + logger.flush() + assert(readFileContent(firstFilePath, codec) === "hello") + + logger.log(" world") + logger.close() + assert(readFileContent(firstFilePath, codec) === "hello world") + } + + /** + * Test logging to multiple files. + */ + private def testMultipleFiles(codecName: Option[String] = None) { + val conf = getLoggingConf(codecName) + val codec = codecName.map { c => CompressionCodec.createCodec(conf) } + val logger = + if (codecName.isDefined) { + new FileLogger(logDirPathString, conf, compress = true) + } else { + new FileLogger(logDirPathString, conf) + } + logger.start() + logger.newFile("Jean_Valjean") + logger.logLine("Who am I?") + logger.logLine("Destiny?") + logger.newFile("John_Valjohn") + logger.logLine("One") + logger.logLine("Two three...") + logger.newFile("Wolverine") + logger.logLine("There was a time") + logger.logLine("A time when our enemies knew honor.") + logger.close() + assert(readFileContent(new Path(logDirPath, "Jean_Valjean"), codec) === "Who am I?\nDestiny?") + assert(readFileContent(new Path(logDirPath, "John_Valjohn"), codec) === "One\nTwo three...") + assert(readFileContent(new Path(logDirPath, "Wolverine"), codec) === + "There was a time\nA time when our enemies knew honor.") + } + + /** + * Read the content of the file specified by the given path. + * If a compression codec is specified, use it to read the file. + */ + private def readFileContent(logPath: Path, codec: Option[CompressionCodec] = None): String = { + val fstream = fileSystem.open(logPath) + val cstream = codec.map(_.compressedInputStream(fstream)).getOrElse(fstream) + Source.fromInputStream(cstream).getLines().mkString("\n") + } + + private def getLoggingConf(codecName: Option[String]) = { + val conf = new SparkConf + codecName.foreach { c => conf.set("spark.io.compression.codec", c) } + conf + } + +} 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 054eb01a64c11..f72389b6b323f 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.util -import java.util.{Properties, UUID} +import java.util.Properties import scala.collection.Map @@ -52,6 +52,8 @@ class JsonProtocolSuite extends FunSuite { val blockManagerRemoved = SparkListenerBlockManagerRemoved( BlockManagerId("Scarce", "to be counted...", 100, 200)) val unpersistRdd = SparkListenerUnpersistRDD(12345) + val applicationStart = SparkListenerApplicationStart("The winner of all", 42L, "Garfield") + val applicationEnd = SparkListenerApplicationEnd(42L) testEvent(stageSubmitted, stageSubmittedJsonString) testEvent(stageCompleted, stageCompletedJsonString) @@ -64,6 +66,8 @@ class JsonProtocolSuite extends FunSuite { testEvent(blockManagerAdded, blockManagerAddedJsonString) testEvent(blockManagerRemoved, blockManagerRemovedJsonString) testEvent(unpersistRdd, unpersistRDDJsonString) + testEvent(applicationStart, applicationStartJsonString) + testEvent(applicationEnd, applicationEndJsonString) } test("Dependent Classes") { @@ -89,7 +93,7 @@ class JsonProtocolSuite extends FunSuite { // JobResult val exception = new Exception("Out of Memory! Please restock film.") exception.setStackTrace(stackTrace) - val jobFailed = JobFailed(exception, 2) + val jobFailed = JobFailed(exception) testJobResult(JobSucceeded) testJobResult(jobFailed) @@ -108,12 +112,22 @@ class JsonProtocolSuite extends FunSuite { // BlockId testBlockId(RDDBlockId(1, 2)) testBlockId(ShuffleBlockId(1, 2, 3)) - testBlockId(BroadcastBlockId(1L)) - testBlockId(BroadcastHelperBlockId(BroadcastBlockId(2L), "Spark")) + testBlockId(BroadcastBlockId(1L, "insert_words_of_wisdom_here")) testBlockId(TaskResultBlockId(1L)) testBlockId(StreamBlockId(1, 2L)) } + test("Backward compatibility") { + // StageInfo.details was added after 1.0.0. + val info = makeStageInfo(1, 2, 3, 4L, 5L) + assert(info.details.nonEmpty) + val newJson = JsonProtocol.stageInfoToJson(info) + val oldJson = newJson.removeField { case (field, _) => field == "Details" } + val newInfo = JsonProtocol.stageInfoFromJson(oldJson) + assert(info.name === newInfo.name) + assert("" === newInfo.details) + } + /** -------------------------- * | Helper test running methods | @@ -209,7 +223,13 @@ class JsonProtocolSuite extends FunSuite { case (e1: SparkListenerBlockManagerRemoved, e2: SparkListenerBlockManagerRemoved) => assertEquals(e1.blockManagerId, e2.blockManagerId) case (e1: SparkListenerUnpersistRDD, e2: SparkListenerUnpersistRDD) => - assert(e1.rddId === e2.rddId) + assert(e1.rddId == e2.rddId) + case (e1: SparkListenerApplicationStart, e2: SparkListenerApplicationStart) => + assert(e1.appName == e2.appName) + assert(e1.time == e2.time) + assert(e1.sparkUser == e2.sparkUser) + case (e1: SparkListenerApplicationEnd, e2: SparkListenerApplicationEnd) => + assert(e1.time == e2.time) case (SparkListenerShutdown, SparkListenerShutdown) => case _ => fail("Events don't match in types!") } @@ -222,7 +242,11 @@ class JsonProtocolSuite extends FunSuite { assert(info1.submissionTime === info2.submissionTime) assert(info1.completionTime === info2.completionTime) assert(info1.emittedTaskSizeWarning === info2.emittedTaskSizeWarning) - assertEquals(info1.rddInfo, info2.rddInfo) + assert(info1.rddInfos.size === info2.rddInfos.size) + (0 until info1.rddInfos.size).foreach { i => + assertEquals(info1.rddInfos(i), info2.rddInfos(i)) + } + assert(info1.details === info2.details) } private def assertEquals(info1: RDDInfo, info2: RDDInfo) { @@ -295,7 +319,6 @@ class JsonProtocolSuite extends FunSuite { (result1, result2) match { case (JobSucceeded, JobSucceeded) => case (r1: JobFailed, r2: JobFailed) => - assert(r1.failedStageId === r2.failedStageId) assertEquals(r1.exception, r2.exception) case _ => fail("Job results don't match in types!") } @@ -426,7 +449,8 @@ class JsonProtocolSuite extends FunSuite { } private def makeStageInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = { - new StageInfo(a, "greetings", b, makeRddInfo(a, b, c, d, e)) + val rddInfos = (1 to a % 5).map { i => makeRddInfo(a % i, b % i, c % i, d % i, e % i) } + new StageInfo(a, "greetings", b, rddInfos, "details") } private def makeTaskInfo(a: Long, b: Int, c: Long) = { @@ -515,8 +539,8 @@ class JsonProtocolSuite extends FunSuite { 700,"Fetch Wait Time":900,"Remote Bytes Read":1000},"Shuffle Write Metrics": {"Shuffle Bytes Written":1200,"Shuffle Write Time":1500},"Updated Blocks": [{"Block ID":{"Type":"RDDBlockId","RDD ID":0,"Split Index":0},"Status": - {"Storage Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":false, - "Replication":2},"Memory Size":0,"Disk Size":0,"Tachyon Size":0}}]}} + {"Storage Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false, + "Deserialized":false,"Replication":2},"Memory Size":0,"Disk Size":0,"Tachyon Size":0}}]}} """ private val jobStartJsonString = @@ -555,4 +579,14 @@ class JsonProtocolSuite extends FunSuite { {"Event":"SparkListenerUnpersistRDD","RDD ID":12345} """ - } + private val applicationStartJsonString = + """ + {"Event":"SparkListenerApplicationStart","App Name":"The winner of all","Timestamp":42, + "User":"Garfield"} + """ + + private val applicationEndJsonString = + """ + {"Event":"SparkListenerApplicationEnd","Timestamp":42} + """ +} 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 e1446cbc90bdb..cf438a3d72a06 100644 --- a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala @@ -22,9 +22,9 @@ import java.util.NoSuchElementException import scala.collection.mutable.Buffer import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers +import org.scalatest.Matchers -class NextIteratorSuite extends FunSuite with ShouldMatchers { +class NextIteratorSuite extends FunSuite with Matchers { test("one iteration") { val i = new StubIterator(Buffer(1)) i.hasNext should be === true @@ -32,7 +32,7 @@ class NextIteratorSuite extends FunSuite with ShouldMatchers { i.hasNext should be === false intercept[NoSuchElementException] { i.next() } } - + test("two iterations") { val i = new StubIterator(Buffer(1, 2)) i.hasNext should be === true @@ -70,7 +70,7 @@ class NextIteratorSuite extends FunSuite with ShouldMatchers { class StubIterator(ints: Buffer[Int]) extends NextIterator[Int] { var closeCalled = 0 - + override def getNext() = { if (ints.size == 0) { finished = true diff --git a/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala new file mode 100644 index 0000000000000..c1c605cdb487b --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala @@ -0,0 +1,265 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.lang.ref.WeakReference + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.util.Random + +import org.scalatest.FunSuite + +class TimeStampedHashMapSuite extends FunSuite { + + // Test the testMap function - a Scala HashMap should obviously pass + testMap(new mutable.HashMap[String, String]()) + + // Test TimeStampedHashMap basic functionality + testMap(new TimeStampedHashMap[String, String]()) + testMapThreadSafety(new TimeStampedHashMap[String, String]()) + + // Test TimeStampedWeakValueHashMap basic functionality + testMap(new TimeStampedWeakValueHashMap[String, String]()) + testMapThreadSafety(new TimeStampedWeakValueHashMap[String, String]()) + + test("TimeStampedHashMap - clearing by timestamp") { + // clearing by insertion time + val map = new TimeStampedHashMap[String, String](updateTimeStampOnGet = false) + map("k1") = "v1" + assert(map("k1") === "v1") + Thread.sleep(10) + val threshTime = System.currentTimeMillis + assert(map.getTimestamp("k1").isDefined) + assert(map.getTimestamp("k1").get < threshTime) + map.clearOldValues(threshTime) + assert(map.get("k1") === None) + + // clearing by modification time + val map1 = new TimeStampedHashMap[String, String](updateTimeStampOnGet = true) + map1("k1") = "v1" + map1("k2") = "v2" + assert(map1("k1") === "v1") + Thread.sleep(10) + val threshTime1 = System.currentTimeMillis + Thread.sleep(10) + assert(map1("k2") === "v2") // access k2 to update its access time to > threshTime + assert(map1.getTimestamp("k1").isDefined) + assert(map1.getTimestamp("k1").get < threshTime1) + assert(map1.getTimestamp("k2").isDefined) + assert(map1.getTimestamp("k2").get >= threshTime1) + map1.clearOldValues(threshTime1) //should only clear k1 + assert(map1.get("k1") === None) + assert(map1.get("k2").isDefined) + } + + test("TimeStampedWeakValueHashMap - clearing by timestamp") { + // clearing by insertion time + val map = new TimeStampedWeakValueHashMap[String, String](updateTimeStampOnGet = false) + map("k1") = "v1" + assert(map("k1") === "v1") + Thread.sleep(10) + val threshTime = System.currentTimeMillis + assert(map.getTimestamp("k1").isDefined) + assert(map.getTimestamp("k1").get < threshTime) + map.clearOldValues(threshTime) + assert(map.get("k1") === None) + + // clearing by modification time + val map1 = new TimeStampedWeakValueHashMap[String, String](updateTimeStampOnGet = true) + map1("k1") = "v1" + map1("k2") = "v2" + assert(map1("k1") === "v1") + Thread.sleep(10) + val threshTime1 = System.currentTimeMillis + Thread.sleep(10) + assert(map1("k2") === "v2") // access k2 to update its access time to > threshTime + assert(map1.getTimestamp("k1").isDefined) + assert(map1.getTimestamp("k1").get < threshTime1) + assert(map1.getTimestamp("k2").isDefined) + assert(map1.getTimestamp("k2").get >= threshTime1) + map1.clearOldValues(threshTime1) //should only clear k1 + assert(map1.get("k1") === None) + assert(map1.get("k2").isDefined) + } + + test("TimeStampedWeakValueHashMap - clearing weak references") { + var strongRef = new Object + val weakRef = new WeakReference(strongRef) + val map = new TimeStampedWeakValueHashMap[String, Object] + map("k1") = strongRef + map("k2") = "v2" + map("k3") = "v3" + val isEquals = map("k1") == strongRef + assert(isEquals) + + // clear strong reference to "k1" + strongRef = null + val startTime = System.currentTimeMillis + System.gc() // Make a best effort to run the garbage collection. It *usually* runs GC. + System.runFinalization() // Make a best effort to call finalizer on all cleaned objects. + while(System.currentTimeMillis - startTime < 10000 && weakRef.get != null) { + System.gc() + System.runFinalization() + Thread.sleep(100) + } + assert(map.getReference("k1").isDefined) + val ref = map.getReference("k1").get + assert(ref.get === null) + assert(map.get("k1") === None) + + // operations should only display non-null entries + assert(map.iterator.forall { case (k, v) => k != "k1" }) + assert(map.filter { case (k, v) => k != "k2" }.size === 1) + assert(map.filter { case (k, v) => k != "k2" }.head._1 === "k3") + assert(map.toMap.size === 2) + assert(map.toMap.forall { case (k, v) => k != "k1" }) + val buffer = new ArrayBuffer[String] + map.foreach { case (k, v) => buffer += v.toString } + assert(buffer.size === 2) + assert(buffer.forall(_ != "k1")) + val plusMap = map + (("k4", "v4")) + assert(plusMap.size === 3) + assert(plusMap.forall { case (k, v) => k != "k1" }) + val minusMap = map - "k2" + assert(minusMap.size === 1) + assert(minusMap.head._1 == "k3") + + // clear null values - should only clear k1 + map.clearNullValues() + assert(map.getReference("k1") === None) + assert(map.get("k1") === None) + assert(map.get("k2").isDefined) + assert(map.get("k2").get === "v2") + } + + /** Test basic operations of a Scala mutable Map. */ + def testMap(hashMapConstructor: => mutable.Map[String, String]) { + def newMap() = hashMapConstructor + val testMap1 = newMap() + val testMap2 = newMap() + val name = testMap1.getClass.getSimpleName + + test(name + " - basic test") { + // put, get, and apply + testMap1 += (("k1", "v1")) + assert(testMap1.get("k1").isDefined) + assert(testMap1.get("k1").get === "v1") + testMap1("k2") = "v2" + assert(testMap1.get("k2").isDefined) + assert(testMap1.get("k2").get === "v2") + assert(testMap1("k2") === "v2") + testMap1.update("k3", "v3") + assert(testMap1.get("k3").isDefined) + assert(testMap1.get("k3").get === "v3") + + // remove + testMap1.remove("k1") + assert(testMap1.get("k1").isEmpty) + testMap1.remove("k2") + intercept[NoSuchElementException] { + testMap1("k2") // Map.apply() causes exception + } + testMap1 -= "k3" + assert(testMap1.get("k3").isEmpty) + + // multi put + val keys = (1 to 100).map(_.toString) + val pairs = keys.map(x => (x, x * 2)) + assert((testMap2 ++ pairs).iterator.toSet === pairs.toSet) + testMap2 ++= pairs + + // iterator + assert(testMap2.iterator.toSet === pairs.toSet) + + // filter + val filtered = testMap2.filter { case (_, v) => v.toInt % 2 == 0 } + val evenPairs = pairs.filter { case (_, v) => v.toInt % 2 == 0 } + assert(filtered.iterator.toSet === evenPairs.toSet) + + // foreach + val buffer = new ArrayBuffer[(String, String)] + testMap2.foreach(x => buffer += x) + assert(testMap2.toSet === buffer.toSet) + + // multi remove + testMap2("k1") = "v1" + testMap2 --= keys + assert(testMap2.size === 1) + assert(testMap2.iterator.toSeq.head === ("k1", "v1")) + + // + + val testMap3 = testMap2 + (("k0", "v0")) + assert(testMap3.size === 2) + assert(testMap3.get("k1").isDefined) + assert(testMap3.get("k1").get === "v1") + assert(testMap3.get("k0").isDefined) + assert(testMap3.get("k0").get === "v0") + + // - + val testMap4 = testMap3 - "k0" + assert(testMap4.size === 1) + assert(testMap4.get("k1").isDefined) + assert(testMap4.get("k1").get === "v1") + } + } + + /** Test thread safety of a Scala mutable map. */ + def testMapThreadSafety(hashMapConstructor: => mutable.Map[String, String]) { + def newMap() = hashMapConstructor + val name = newMap().getClass.getSimpleName + val testMap = newMap() + @volatile var error = false + + def getRandomKey(m: mutable.Map[String, String]): Option[String] = { + val keys = testMap.keysIterator.toSeq + if (keys.nonEmpty) { + Some(keys(Random.nextInt(keys.size))) + } else { + None + } + } + + val threads = (1 to 25).map(i => new Thread() { + override def run() { + try { + for (j <- 1 to 1000) { + Random.nextInt(3) match { + case 0 => + testMap(Random.nextString(10)) = Random.nextDouble().toString // put + case 1 => + getRandomKey(testMap).map(testMap.get) // get + case 2 => + getRandomKey(testMap).map(testMap.remove) // remove + } + } + } catch { + case t: Throwable => + error = true + throw t + } + } + }) + + test(name + " - threading safety test") { + threads.map(_.start) + threads.map(_.join) + assert(!error) + } + } +} 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 616214fb5e3a6..1ee936bc78f49 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.util import scala.util.Random -import java.io.{ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream} +import java.io.{File, ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream} +import java.net.URI import java.nio.{ByteBuffer, ByteOrder} import com.google.common.base.Charsets @@ -112,6 +113,7 @@ class UtilsSuite extends FunSuite { test("reading offset bytes of a file") { val tmpDir2 = Files.createTempDir() + tmpDir2.deleteOnExit() val f1Path = tmpDir2 + "/f1" val f1 = new FileOutputStream(f1Path) f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(Charsets.UTF_8)) @@ -138,6 +140,38 @@ class UtilsSuite extends FunSuite { Utils.deleteRecursively(tmpDir2) } + test("reading offset bytes across multiple files") { + val tmpDir = Files.createTempDir() + tmpDir.deleteOnExit() + val files = (1 to 3).map(i => new File(tmpDir, i.toString)) + Files.write("0123456789", files(0), Charsets.UTF_8) + Files.write("abcdefghij", files(1), Charsets.UTF_8) + Files.write("ABCDEFGHIJ", files(2), Charsets.UTF_8) + + // Read first few bytes in the 1st file + assert(Utils.offsetBytes(files, 0, 5) === "01234") + + // Read bytes within the 1st file + assert(Utils.offsetBytes(files, 5, 8) === "567") + + // Read bytes across 1st and 2nd file + assert(Utils.offsetBytes(files, 8, 18) === "89abcdefgh") + + // Read bytes across 1st, 2nd and 3rd file + assert(Utils.offsetBytes(files, 5, 24) === "56789abcdefghijABCD") + + // Read some nonexistent bytes in the beginning + assert(Utils.offsetBytes(files, -5, 18) === "0123456789abcdefgh") + + // Read some nonexistent bytes at the end + assert(Utils.offsetBytes(files, 18, 35) === "ijABCDEFGHIJ") + + // Read some nonexistent bytes on both ends + assert(Utils.offsetBytes(files, -5, 35) === "0123456789abcdefghijABCDEFGHIJ") + + Utils.deleteRecursively(tmpDir) + } + test("deserialize long value") { val testval : Long = 9730889947L val bbuf = ByteBuffer.allocate(8) @@ -154,5 +188,81 @@ class UtilsSuite extends FunSuite { val iterator = Iterator.range(0, 5) assert(Utils.getIteratorSize(iterator) === 5L) } -} + test("findOldFiles") { + // 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 + val child2: File = Utils.createTempDir(parent.getCanonicalPath) + // set the last modified time of child1 to 10 secs old + child1.setLastModified(System.currentTimeMillis() - (1000 * 10)) + + val result = Utils.findOldFiles(parent, 5) // find files older than 5 secs + assert(result.size.equals(1)) + assert(result(0).getCanonicalPath.equals(child1.getCanonicalPath)) + } + + test("resolveURI") { + def assertResolves(before: String, after: String, testWindows: Boolean = false): Unit = { + assume(before.split(",").length == 1) + assert(Utils.resolveURI(before, testWindows) === new URI(after)) + assert(Utils.resolveURI(after, testWindows) === new URI(after)) + assert(new URI(Utils.resolveURIs(before, testWindows)) === new URI(after)) + assert(new URI(Utils.resolveURIs(after, testWindows)) === new URI(after)) + } + val cwd = System.getProperty("user.dir") + assertResolves("hdfs:/root/spark.jar", "hdfs:/root/spark.jar") + assertResolves("hdfs:///root/spark.jar#app.jar", "hdfs:/root/spark.jar#app.jar") + assertResolves("spark.jar", s"file:$cwd/spark.jar") + assertResolves("spark.jar#app.jar", s"file:$cwd/spark.jar#app.jar") + assertResolves("C:/path/to/file.txt", "file:/C:/path/to/file.txt", testWindows = true) + assertResolves("C:\\path\\to\\file.txt", "file:/C:/path/to/file.txt", testWindows = true) + assertResolves("file:/C:/path/to/file.txt", "file:/C:/path/to/file.txt", testWindows = true) + assertResolves("file:///C:/path/to/file.txt", "file:/C:/path/to/file.txt", testWindows = true) + assertResolves("file:/C:/file.txt#alias.txt", "file:/C:/file.txt#alias.txt", testWindows = true) + intercept[IllegalArgumentException] { Utils.resolveURI("file:foo") } + intercept[IllegalArgumentException] { Utils.resolveURI("file:foo:baby") } + + // Test resolving comma-delimited paths + assert(Utils.resolveURIs("jar1,jar2") === s"file:$cwd/jar1,file:$cwd/jar2") + assert(Utils.resolveURIs("file:/jar1,file:/jar2") === "file:/jar1,file:/jar2") + assert(Utils.resolveURIs("hdfs:/jar1,file:/jar2,jar3") === + s"hdfs:/jar1,file:/jar2,file:$cwd/jar3") + assert(Utils.resolveURIs("hdfs:/jar1,file:/jar2,jar3,jar4#jar5") === + s"hdfs:/jar1,file:/jar2,file:$cwd/jar3,file:$cwd/jar4#jar5") + assert(Utils.resolveURIs("hdfs:/jar1,file:/jar2,jar3,C:\\pi.py#py.pi", testWindows = true) === + s"hdfs:/jar1,file:/jar2,file:$cwd/jar3,file:/C:/pi.py#py.pi") + } + + test("nonLocalPaths") { + assert(Utils.nonLocalPaths("spark.jar") === Array.empty) + assert(Utils.nonLocalPaths("file:/spark.jar") === Array.empty) + assert(Utils.nonLocalPaths("file:///spark.jar") === Array.empty) + assert(Utils.nonLocalPaths("local:/spark.jar") === Array.empty) + assert(Utils.nonLocalPaths("local:///spark.jar") === Array.empty) + assert(Utils.nonLocalPaths("hdfs:/spark.jar") === Array("hdfs:/spark.jar")) + assert(Utils.nonLocalPaths("hdfs:///spark.jar") === Array("hdfs:///spark.jar")) + assert(Utils.nonLocalPaths("file:/spark.jar,local:/smart.jar,family.py") === Array.empty) + assert(Utils.nonLocalPaths("local:/spark.jar,file:/smart.jar,family.py") === Array.empty) + assert(Utils.nonLocalPaths("hdfs:/spark.jar,s3:/smart.jar") === + Array("hdfs:/spark.jar", "s3:/smart.jar")) + assert(Utils.nonLocalPaths("hdfs:/spark.jar,s3:/smart.jar,local.py,file:/hello/pi.py") === + Array("hdfs:/spark.jar", "s3:/smart.jar")) + assert(Utils.nonLocalPaths("local.py,hdfs:/spark.jar,file:/hello/pi.py,s3:/smart.jar") === + Array("hdfs:/spark.jar", "s3:/smart.jar")) + + // Test Windows paths + assert(Utils.nonLocalPaths("C:/some/path.jar", testWindows = true) === Array.empty) + assert(Utils.nonLocalPaths("file:/C:/some/path.jar", testWindows = true) === Array.empty) + assert(Utils.nonLocalPaths("file:///C:/some/path.jar", testWindows = true) === Array.empty) + assert(Utils.nonLocalPaths("local:/C:/some/path.jar", testWindows = true) === Array.empty) + assert(Utils.nonLocalPaths("local:///C:/some/path.jar", testWindows = true) === Array.empty) + assert(Utils.nonLocalPaths("hdfs:/a.jar,C:/my.jar,s3:/another.jar", testWindows = true) === + Array("hdfs:/a.jar", "s3:/another.jar")) + assert(Utils.nonLocalPaths("D:/your.jar,hdfs:/a.jar,s3:/another.jar", testWindows = true) === + Array("hdfs:/a.jar", "s3:/another.jar")) + assert(Utils.nonLocalPaths("hdfs:/a.jar,s3:/another.jar,e:/our.jar", testWindows = true) === + Array("hdfs:/a.jar", "s3:/another.jar")) + } + +} 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 fce1184d46364..deb780953579d 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 @@ -174,9 +174,9 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { assert(result1.toSet == Set[(Int, Int)]((0, 5), (1, 5))) // groupByKey - val result2 = rdd.groupByKey().collect() + val result2 = rdd.groupByKey().collect().map(x => (x._1, x._2.toList)).toSet assert(result2.toSet == Set[(Int, Seq[Int])] - ((0, ArrayBuffer[Int](1, 1, 1, 1, 1)), (1, ArrayBuffer[Int](1, 1, 1, 1, 1)))) + ((0, List[Int](1, 1, 1, 1, 1)), (1, List[Int](1, 1, 1, 1, 1)))) } test("simple cogroup") { @@ -277,6 +277,11 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { ("pomatoes", "eructation") // 568647356 ) + collisionPairs.foreach { case (w1, w2) => + // String.hashCode is documented to use a specific algorithm, but check just in case + assert(w1.hashCode === w2.hashCode) + } + (1 to 100000).map(_.toString).foreach { i => map.insert(i, i) } collisionPairs.foreach { case (w1, w2) => map.insert(w1, w2) @@ -296,7 +301,32 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { assert(kv._2.equals(expectedValue)) count += 1 } - assert(count == 100000 + collisionPairs.size * 2) + assert(count === 100000 + collisionPairs.size * 2) + } + + test("spilling with many hash collisions") { + val conf = new SparkConf(true) + conf.set("spark.shuffle.memoryFraction", "0.0001") + sc = new SparkContext("local-cluster[1,1,512]", "test", conf) + + val map = new ExternalAppendOnlyMap[FixedHashObject, Int, Int](_ => 1, _ + _, _ + _) + + // Insert 10 copies each of lots of objects whose hash codes are either 0 or 1. This causes + // problems if the map fails to group together the objects with the same code (SPARK-2043). + for (i <- 1 to 10) { + for (j <- 1 to 10000) { + map.insert(FixedHashObject(j, j % 2), 1) + } + } + + val it = map.iterator + var count = 0 + while (it.hasNext) { + val kv = it.next() + assert(kv._2 === 10) + count += 1 + } + assert(count === 10000) } test("spilling with hash collisions using the Int.MaxValue key") { @@ -317,3 +347,10 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } } } + +/** + * A dummy class that always returns the same hash code, to easily test hash collisions + */ +case class FixedHashObject(val v: Int, val h: Int) extends Serializable { + override def hashCode(): Int = h +} 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 b024c89d94d33..6a70877356409 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 @@ -20,11 +20,11 @@ package org.apache.spark.util.collection import scala.collection.mutable.HashSet import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers +import org.scalatest.Matchers import org.apache.spark.util.SizeEstimator -class OpenHashMapSuite extends FunSuite with ShouldMatchers { +class OpenHashMapSuite extends FunSuite with Matchers { test("size for specialized, primitive value (int)") { val capacity = 1024 diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala index ff4a98f5dcd4a..68a03e3a0970f 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala @@ -18,11 +18,11 @@ package org.apache.spark.util.collection import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers +import org.scalatest.Matchers import org.apache.spark.util.SizeEstimator -class OpenHashSetSuite extends FunSuite with ShouldMatchers { +class OpenHashSetSuite extends FunSuite with Matchers { test("size for specialized, primitive int") { val loadFactor = 0.7 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 e3fca173908e9..8c7df7d73dcd3 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 @@ -20,11 +20,11 @@ package org.apache.spark.util.collection import scala.collection.mutable.HashSet import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers +import org.scalatest.Matchers import org.apache.spark.util.SizeEstimator -class PrimitiveKeyOpenHashMapSuite extends FunSuite with ShouldMatchers { +class PrimitiveKeyOpenHashMapSuite extends FunSuite with Matchers { test("size for specialized, primitive key, value (int, int)") { val capacity = 1024 diff --git a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala index 7576c9a51f313..e166787f17544 100644 --- a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala @@ -41,21 +41,31 @@ class RandomSamplerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar random.nextDouble().andReturn(x) } } - whenExecuting(random) - { + whenExecuting(random) { val sampler = new BernoulliSampler[Int](0.25, 0.55)(random) assert(sampler.sample(a.iterator).toList == List(3, 4, 5)) } } + test("BernoulliSamplerWithRangeInverse") { + expecting { + for(x <- Seq(0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9)) { + random.nextDouble().andReturn(x) + } + } + whenExecuting(random) { + val sampler = new BernoulliSampler[Int](0.25, 0.55, true)(random) + assert(sampler.sample(a.iterator).toList === List(1, 2, 6, 7, 8, 9)) + } + } + test("BernoulliSamplerWithRatio") { expecting { for(x <- Seq(0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9)) { random.nextDouble().andReturn(x) } } - whenExecuting(random) - { + whenExecuting(random) { val sampler = new BernoulliSampler[Int](0.35)(random) assert(sampler.sample(a.iterator).toList == List(1, 2, 3)) } @@ -67,8 +77,7 @@ class RandomSamplerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar random.nextDouble().andReturn(x) } } - whenExecuting(random) - { + whenExecuting(random) { val sampler = new BernoulliSampler[Int](0.25, 0.55, true)(random) assert(sampler.sample(a.iterator).toList == List(1, 2, 6, 7, 8, 9)) } @@ -78,8 +87,7 @@ class RandomSamplerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar expecting { random.setSeed(10L) } - whenExecuting(random) - { + whenExecuting(random) { val sampler = new BernoulliSampler[Int](0.2)(random) sampler.setSeed(10L) } diff --git a/core/src/test/scala/org/apache/spark/util/random/SamplingUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/random/SamplingUtilsSuite.scala new file mode 100644 index 0000000000000..accfe2e9b7f2a --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/random/SamplingUtilsSuite.scala @@ -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. + */ + +package org.apache.spark.util.random + +import org.apache.commons.math3.distribution.{BinomialDistribution, PoissonDistribution} +import org.scalatest.FunSuite + +class SamplingUtilsSuite extends FunSuite { + + test("computeFraction") { + // test that the computed fraction guarantees enough data points + // in the sample with a failure rate <= 0.0001 + val n = 100000 + + for (s <- 1 to 15) { + val frac = SamplingUtils.computeFractionForSampleSize(s, n, true) + val poisson = new PoissonDistribution(frac * n) + assert(poisson.inverseCumulativeProbability(0.0001) >= s, "Computed fraction is too low") + } + for (s <- List(20, 100, 1000)) { + val frac = SamplingUtils.computeFractionForSampleSize(s, n, true) + val poisson = new PoissonDistribution(frac * n) + assert(poisson.inverseCumulativeProbability(0.0001) >= s, "Computed fraction is too low") + } + for (s <- List(1, 10, 100, 1000)) { + val frac = SamplingUtils.computeFractionForSampleSize(s, n, false) + val binomial = new BinomialDistribution(n, frac) + assert(binomial.inverseCumulativeProbability(0.0001)*n >= s, "Computed fraction is too low") + } + } +} 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 757476efdb789..ef7178bcdf5c2 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 @@ -18,58 +18,47 @@ package org.apache.spark.util.random import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers +import org.scalatest.Matchers + +import org.apache.commons.math3.stat.inference.ChiSquareTest import org.apache.spark.util.Utils.times -class XORShiftRandomSuite extends FunSuite with ShouldMatchers { +import scala.language.reflectiveCalls + +class XORShiftRandomSuite extends FunSuite with Matchers { def fixture = new { val seed = 1L val xorRand = new XORShiftRandom(seed) val hundMil = 1e8.toInt } - + /* - * This test is based on a chi-squared test for randomness. The values are hard-coded - * so as not to create Spark's dependency on apache.commons.math3 just to call one - * method for calculating the exact p-value for a given number of random numbers - * and bins. In case one would want to move to a full-fledged test based on - * apache.commons.math3, the relevant class is here: - * org.apache.commons.math3.stat.inference.ChiSquareTest + * This test is based on a chi-squared test for randomness. */ test ("XORShift generates valid random numbers") { val f = fixture - val numBins = 10 - // create 10 bins - val bins = Array.fill(numBins)(0) + val numBins = 10 // create 10 bins + val numRows = 5 // create 5 rows + val bins = Array.ofDim[Long](numRows, numBins) - // populate bins based on modulus of the random number - times(f.hundMil) {bins(math.abs(f.xorRand.nextInt) % 10) += 1} + // populate bins based on modulus of the random number for each row + for (r <- 0 to numRows-1) { + times(f.hundMil) {bins(r)(math.abs(f.xorRand.nextInt) % numBins) += 1} + } - /* since the seed is deterministic, until the algorithm is changed, we know the result will be - * exactly this: Array(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, - * 10000790, 10002286, 9998699), so the test will never fail at the prespecified (5%) - * significance level. However, should the RNG implementation change, the test should still - * pass at the same significance level. The chi-squared test done in R gave the following - * results: - * > chisq.test(c(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, - * 10000790, 10002286, 9998699)) - * Chi-squared test for given probabilities - * data: c(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, 10000790, - * 10002286, 9998699) - * X-squared = 11.975, df = 9, p-value = 0.2147 - * Note that the p-value was ~0.22. The test will fail if alpha < 0.05, which for 100 million - * random numbers - * and 10 bins will happen at X-squared of ~16.9196. So, the test will fail if X-squared - * is greater than or equal to that number. + /* + * Perform the chi square test on the 5 rows of randomly generated numbers evenly divided into + * 10 bins. chiSquareTest returns true iff the null hypothesis (that the classifications + * represented by the counts in the columns of the input 2-way table are independent of the + * rows) can be rejected with 100 * (1 - alpha) percent confidence, where alpha is prespeficied + * as 0.05 */ - val binSize = f.hundMil/numBins - val xSquared = bins.map(x => math.pow((binSize - x), 2)/binSize).sum - xSquared should be < (16.9196) - + val chiTest = new ChiSquareTest + assert(chiTest.chiSquareTest(bins, 0.05) === false) } test ("XORShift with zero seed") { diff --git a/data/mllib/sample_binary_classification_data.txt b/data/mllib/sample_binary_classification_data.txt new file mode 100644 index 0000000000000..861c70cde7fd2 --- /dev/null +++ b/data/mllib/sample_binary_classification_data.txt @@ -0,0 +1,100 @@ +0 128:51 129:159 130:253 131:159 132:50 155:48 156:238 157:252 158:252 159:252 160:237 182:54 183:227 184:253 185:252 186:239 187:233 188:252 189:57 190:6 208:10 209:60 210:224 211:252 212:253 213:252 214:202 215:84 216:252 217:253 218:122 236:163 237:252 238:252 239:252 240:253 241:252 242:252 243:96 244:189 245:253 246:167 263:51 264:238 265:253 266:253 267:190 268:114 269:253 270:228 271:47 272:79 273:255 274:168 290:48 291:238 292:252 293:252 294:179 295:12 296:75 297:121 298:21 301:253 302:243 303:50 317:38 318:165 319:253 320:233 321:208 322:84 329:253 330:252 331:165 344:7 345:178 346:252 347:240 348:71 349:19 350:28 357:253 358:252 359:195 372:57 373:252 374:252 375:63 385:253 386:252 387:195 400:198 401:253 402:190 413:255 414:253 415:196 427:76 428:246 429:252 430:112 441:253 442:252 443:148 455:85 456:252 457:230 458:25 467:7 468:135 469:253 470:186 471:12 483:85 484:252 485:223 494:7 495:131 496:252 497:225 498:71 511:85 512:252 513:145 521:48 522:165 523:252 524:173 539:86 540:253 541:225 548:114 549:238 550:253 551:162 567:85 568:252 569:249 570:146 571:48 572:29 573:85 574:178 575:225 576:253 577:223 578:167 579:56 595:85 596:252 597:252 598:252 599:229 600:215 601:252 602:252 603:252 604:196 605:130 623:28 624:199 625:252 626:252 627:253 628:252 629:252 630:233 631:145 652:25 653:128 654:252 655:253 656:252 657:141 658:37 +1 159:124 160:253 161:255 162:63 186:96 187:244 188:251 189:253 190:62 214:127 215:251 216:251 217:253 218:62 241:68 242:236 243:251 244:211 245:31 246:8 268:60 269:228 270:251 271:251 272:94 296:155 297:253 298:253 299:189 323:20 324:253 325:251 326:235 327:66 350:32 351:205 352:253 353:251 354:126 378:104 379:251 380:253 381:184 382:15 405:80 406:240 407:251 408:193 409:23 432:32 433:253 434:253 435:253 436:159 460:151 461:251 462:251 463:251 464:39 487:48 488:221 489:251 490:251 491:172 515:234 516:251 517:251 518:196 519:12 543:253 544:251 545:251 546:89 570:159 571:255 572:253 573:253 574:31 597:48 598:228 599:253 600:247 601:140 602:8 625:64 626:251 627:253 628:220 653:64 654:251 655:253 656:220 681:24 682:193 683:253 684:220 +1 125:145 126:255 127:211 128:31 152:32 153:237 154:253 155:252 156:71 180:11 181:175 182:253 183:252 184:71 209:144 210:253 211:252 212:71 236:16 237:191 238:253 239:252 240:71 264:26 265:221 266:253 267:252 268:124 269:31 293:125 294:253 295:252 296:252 297:108 322:253 323:252 324:252 325:108 350:255 351:253 352:253 353:108 378:253 379:252 380:252 381:108 406:253 407:252 408:252 409:108 434:253 435:252 436:252 437:108 462:255 463:253 464:253 465:170 490:253 491:252 492:252 493:252 494:42 518:149 519:252 520:252 521:252 522:144 546:109 547:252 548:252 549:252 550:144 575:218 576:253 577:253 578:255 579:35 603:175 604:252 605:252 606:253 607:35 631:73 632:252 633:252 634:253 635:35 659:31 660:211 661:252 662:253 663:35 +1 153:5 154:63 155:197 181:20 182:254 183:230 184:24 209:20 210:254 211:254 212:48 237:20 238:254 239:255 240:48 265:20 266:254 267:254 268:57 293:20 294:254 295:254 296:108 321:16 322:239 323:254 324:143 350:178 351:254 352:143 378:178 379:254 380:143 406:178 407:254 408:162 434:178 435:254 436:240 462:113 463:254 464:240 490:83 491:254 492:245 493:31 518:79 519:254 520:246 521:38 547:214 548:254 549:150 575:144 576:241 577:8 603:144 604:240 605:2 631:144 632:254 633:82 659:230 660:247 661:40 687:168 688:209 689:31 +1 152:1 153:168 154:242 155:28 180:10 181:228 182:254 183:100 209:190 210:254 211:122 237:83 238:254 239:162 265:29 266:254 267:248 268:25 293:29 294:255 295:254 296:103 321:29 322:254 323:254 324:109 349:29 350:254 351:254 352:109 377:29 378:254 379:254 380:109 405:29 406:255 407:254 408:109 433:29 434:254 435:254 436:109 461:29 462:254 463:254 464:63 489:29 490:254 491:254 492:28 517:29 518:254 519:254 520:28 545:29 546:254 547:254 548:35 573:29 574:254 575:254 576:109 601:6 602:212 603:254 604:109 630:203 631:254 632:178 658:155 659:254 660:190 686:32 687:199 688:104 +0 130:64 131:253 132:255 133:63 157:96 158:205 159:251 160:253 161:205 162:111 163:4 184:96 185:189 186:251 187:251 188:253 189:251 190:251 191:31 209:16 210:64 211:223 212:244 213:251 214:251 215:211 216:213 217:251 218:251 219:31 236:80 237:181 238:251 239:253 240:251 241:251 242:251 243:94 244:96 245:251 246:251 247:31 263:92 264:253 265:253 266:253 267:255 268:253 269:253 270:253 271:95 272:96 273:253 274:253 275:31 290:92 291:236 292:251 293:243 294:220 295:233 296:251 297:251 298:243 299:82 300:96 301:251 302:251 303:31 317:80 318:253 319:251 320:251 321:188 323:96 324:251 325:251 326:109 328:96 329:251 330:251 331:31 344:96 345:240 346:253 347:243 348:188 349:42 351:96 352:204 353:109 354:4 356:12 357:197 358:251 359:31 372:221 373:251 374:253 375:121 379:36 380:23 385:190 386:251 387:31 399:48 400:234 401:253 413:191 414:253 415:31 426:44 427:221 428:251 429:251 440:12 441:197 442:251 443:31 454:190 455:251 456:251 457:251 468:96 469:251 470:251 471:31 482:190 483:251 484:251 485:113 495:40 496:234 497:251 498:219 499:23 510:190 511:251 512:251 513:94 522:40 523:217 524:253 525:231 526:47 538:191 539:253 540:253 541:253 548:12 549:174 550:253 551:253 552:219 553:39 566:67 567:236 568:251 569:251 570:191 571:190 572:111 573:72 574:190 575:191 576:197 577:251 578:243 579:121 580:39 595:63 596:236 597:251 598:253 599:251 600:251 601:251 602:251 603:253 604:251 605:188 606:94 624:27 625:129 626:253 627:251 628:251 629:251 630:251 631:229 632:168 633:15 654:95 655:212 656:251 657:211 658:94 659:59 +1 159:121 160:254 161:136 186:13 187:230 188:253 189:248 190:99 213:4 214:118 215:253 216:253 217:225 218:42 241:61 242:253 243:253 244:253 245:74 268:32 269:206 270:253 271:253 272:186 273:9 296:211 297:253 298:253 299:239 300:69 324:254 325:253 326:253 327:133 351:142 352:255 353:253 354:186 355:8 378:149 379:229 380:254 381:207 382:21 405:54 406:229 407:253 408:254 409:105 433:152 434:254 435:254 436:213 437:26 460:112 461:251 462:253 463:253 464:26 487:29 488:212 489:253 490:250 491:149 514:36 515:214 516:253 517:253 518:137 542:75 543:253 544:253 545:253 546:59 570:93 571:253 572:253 573:189 574:17 598:224 599:253 600:253 601:84 625:43 626:235 627:253 628:126 629:1 653:99 654:248 655:253 656:119 682:225 683:235 684:49 +1 100:166 101:222 102:55 128:197 129:254 130:218 131:5 155:29 156:249 157:254 158:254 159:9 183:45 184:254 185:254 186:174 187:2 210:4 211:164 212:254 213:254 214:85 238:146 239:254 240:254 241:254 242:85 265:101 266:245 267:254 268:254 269:254 270:85 292:97 293:248 294:254 295:204 296:254 297:254 298:85 315:12 316:59 317:98 318:151 319:237 320:254 321:254 322:109 323:35 324:254 325:254 326:85 343:41 344:216 345:254 346:254 347:239 348:153 349:37 350:4 351:32 352:254 353:254 354:85 372:7 373:44 374:44 375:30 379:32 380:254 381:254 382:96 407:19 408:230 409:254 410:174 436:197 437:254 438:110 464:197 465:254 466:85 492:197 493:253 494:63 515:37 516:54 517:54 518:45 519:26 520:84 521:221 522:84 523:21 524:31 525:162 526:78 540:6 541:41 542:141 543:244 544:254 545:254 546:248 547:236 548:254 549:254 550:254 551:233 552:239 553:254 554:138 567:23 568:167 569:254 570:254 571:254 572:254 573:229 574:228 575:185 576:138 577:138 578:138 579:138 580:138 581:138 582:44 595:113 596:254 597:254 598:254 599:179 600:64 601:5 623:32 624:209 625:183 626:97 +0 155:53 156:255 157:253 158:253 159:253 160:124 183:180 184:253 185:251 186:251 187:251 188:251 189:145 190:62 209:32 210:217 211:241 212:253 213:251 214:251 215:251 216:251 217:253 218:107 237:37 238:251 239:251 240:253 241:251 242:251 243:251 244:251 245:253 246:107 265:166 266:251 267:251 268:253 269:251 270:96 271:148 272:251 273:253 274:107 291:73 292:253 293:253 294:253 295:253 296:130 299:110 300:253 301:255 302:108 319:73 320:251 321:251 322:251 323:251 327:109 328:251 329:253 330:107 347:202 348:251 349:251 350:251 351:225 354:6 355:129 356:251 357:253 358:107 375:150 376:251 377:251 378:251 379:71 382:115 383:251 384:251 385:253 386:107 403:253 404:251 405:251 406:173 407:20 410:217 411:251 412:251 413:253 414:107 430:182 431:255 432:253 433:216 438:218 439:253 440:253 441:182 457:63 458:221 459:253 460:251 461:215 465:84 466:236 467:251 468:251 469:77 485:109 486:251 487:253 488:251 489:215 492:11 493:160 494:251 495:251 496:96 513:109 514:251 515:253 516:251 517:137 520:150 521:251 522:251 523:251 524:71 541:109 542:251 543:253 544:251 545:35 547:130 548:253 549:251 550:251 551:173 552:20 569:110 570:253 571:255 572:253 573:98 574:150 575:253 576:255 577:253 578:164 597:109 598:251 599:253 600:251 601:251 602:251 603:251 604:253 605:251 606:35 625:93 626:241 627:253 628:251 629:251 630:251 631:251 632:216 633:112 634:5 654:103 655:253 656:251 657:251 658:251 659:251 683:124 684:251 685:225 686:71 687:71 +0 128:73 129:253 130:227 131:73 132:21 156:73 157:251 158:251 159:251 160:174 182:16 183:166 184:228 185:251 186:251 187:251 188:122 210:62 211:220 212:253 213:251 214:251 215:251 216:251 217:79 238:79 239:231 240:253 241:251 242:251 243:251 244:251 245:232 246:77 264:145 265:253 266:253 267:253 268:255 269:253 270:253 271:253 272:253 273:255 274:108 292:144 293:251 294:251 295:251 296:253 297:168 298:107 299:169 300:251 301:253 302:189 303:20 318:27 319:89 320:236 321:251 322:235 323:215 324:164 325:15 326:6 327:129 328:251 329:253 330:251 331:35 345:47 346:211 347:253 348:251 349:251 350:142 354:37 355:251 356:251 357:253 358:251 359:35 373:109 374:251 375:253 376:251 377:251 378:142 382:11 383:148 384:251 385:253 386:251 387:164 400:11 401:150 402:253 403:255 404:211 405:25 410:11 411:150 412:253 413:255 414:211 415:25 428:140 429:251 430:251 431:253 432:107 438:37 439:251 440:251 441:211 442:46 456:190 457:251 458:251 459:253 460:128 461:5 466:37 467:251 468:251 469:51 484:115 485:251 486:251 487:253 488:188 489:20 492:32 493:109 494:129 495:251 496:173 497:103 512:217 513:251 514:251 515:201 516:30 520:73 521:251 522:251 523:251 524:71 540:166 541:253 542:253 543:255 544:149 545:73 546:150 547:253 548:255 549:253 550:253 551:143 568:140 569:251 570:251 571:253 572:251 573:251 574:251 575:251 576:253 577:251 578:230 579:61 596:190 597:251 598:251 599:253 600:251 601:251 602:251 603:251 604:242 605:215 606:55 624:21 625:189 626:251 627:253 628:251 629:251 630:251 631:173 632:103 653:31 654:200 655:253 656:251 657:96 658:71 659:20 +1 155:178 156:255 157:105 182:6 183:188 184:253 185:216 186:14 210:14 211:202 212:253 213:253 214:23 238:12 239:199 240:253 241:128 242:6 266:42 267:253 268:253 269:158 294:42 295:253 296:253 297:158 322:155 323:253 324:253 325:158 350:160 351:253 352:253 353:147 378:160 379:253 380:253 381:41 405:17 406:225 407:253 408:235 409:31 433:24 434:253 435:253 436:176 461:24 462:253 463:253 464:176 489:24 490:253 491:253 492:176 517:24 518:253 519:253 520:176 545:24 546:253 547:253 548:162 573:46 574:253 575:253 576:59 601:142 602:253 603:253 604:59 629:142 630:253 631:253 632:59 657:142 658:253 659:202 660:8 685:87 686:253 687:139 +0 154:46 155:105 156:254 157:254 158:254 159:254 160:255 161:239 162:41 180:37 181:118 182:222 183:254 184:253 185:253 186:253 187:253 188:253 189:253 190:211 191:54 207:14 208:200 209:253 210:253 211:254 212:253 213:253 214:253 215:253 216:253 217:253 218:253 219:116 233:16 234:160 235:236 236:253 237:253 238:253 239:254 240:253 241:253 242:246 243:229 244:253 245:253 246:253 247:116 261:99 262:253 263:253 264:253 265:253 266:253 267:254 268:253 269:253 270:213 271:99 272:253 273:253 274:253 275:116 288:25 289:194 290:253 291:253 292:253 293:253 294:131 295:97 296:169 297:253 298:93 299:99 300:253 301:253 302:253 303:116 316:206 317:253 318:253 319:251 320:233 321:127 322:9 324:18 325:38 326:3 327:15 328:171 329:253 330:253 331:116 343:55 344:240 345:253 346:253 347:233 355:31 356:186 357:253 358:253 359:116 371:176 372:253 373:253 374:253 375:127 383:99 384:253 385:253 386:253 387:116 399:176 400:253 401:253 402:131 403:9 411:99 412:253 413:253 414:253 415:116 426:119 427:254 428:254 429:232 430:75 440:158 441:254 442:254 443:117 454:118 455:253 456:253 457:154 468:156 469:253 470:253 471:116 482:118 483:253 484:253 485:154 496:156 497:253 498:253 499:116 509:46 510:222 511:253 512:253 513:154 522:7 523:116 524:246 525:253 526:180 527:9 538:118 539:253 540:253 541:154 550:116 551:253 552:253 553:253 554:174 566:118 567:253 568:253 569:154 577:110 578:246 579:253 580:253 581:240 582:67 594:118 595:253 596:253 597:238 598:215 599:49 600:20 601:20 602:20 603:66 604:215 605:241 606:253 607:245 608:233 609:64 622:82 623:229 624:253 625:253 626:253 627:253 628:253 629:253 630:253 631:254 632:253 633:253 634:240 635:107 651:176 652:253 653:253 654:253 655:253 656:253 657:253 658:253 659:254 660:253 661:253 662:108 679:40 680:239 681:253 682:253 683:253 684:253 685:253 686:253 687:254 688:161 689:57 690:4 +0 152:56 153:105 154:220 155:254 156:63 178:18 179:166 180:233 181:253 182:253 183:253 184:236 185:209 186:209 187:209 188:77 189:18 206:84 207:253 208:253 209:253 210:253 211:253 212:254 213:253 214:253 215:253 216:253 217:172 218:8 233:57 234:238 235:253 236:253 237:253 238:253 239:253 240:254 241:253 242:253 243:253 244:253 245:253 246:119 260:14 261:238 262:253 263:253 264:253 265:253 266:253 267:253 268:179 269:196 270:253 271:253 272:253 273:253 274:238 275:12 288:33 289:253 290:253 291:253 292:253 293:253 294:248 295:134 297:18 298:83 299:237 300:253 301:253 302:253 303:14 316:164 317:253 318:253 319:253 320:253 321:253 322:128 327:57 328:119 329:214 330:253 331:94 343:57 344:248 345:253 346:253 347:253 348:126 349:14 350:4 357:179 358:253 359:248 360:56 371:175 372:253 373:253 374:240 375:190 376:28 385:179 386:253 387:253 388:173 399:209 400:253 401:253 402:178 413:92 414:253 415:253 416:208 427:211 428:254 429:254 430:179 442:135 443:255 444:209 455:209 456:253 457:253 458:90 470:134 471:253 472:208 483:209 484:253 485:253 486:178 497:2 498:142 499:253 500:208 511:209 512:253 513:253 514:214 515:35 525:30 526:253 527:253 528:208 539:165 540:253 541:253 542:253 543:215 544:36 553:163 554:253 555:253 556:164 567:18 568:172 569:253 570:253 571:253 572:214 573:127 574:7 580:72 581:232 582:253 583:171 584:17 596:8 597:182 598:253 599:253 600:253 601:253 602:162 603:56 607:64 608:240 609:253 610:253 611:14 625:7 626:173 627:253 628:253 629:253 630:253 631:245 632:241 633:239 634:239 635:246 636:253 637:225 638:14 639:1 654:18 655:59 656:138 657:224 658:253 659:253 660:254 661:253 662:253 663:253 664:240 665:96 685:37 686:104 687:192 688:255 689:253 690:253 691:182 692:73 +1 130:7 131:176 132:254 133:224 158:51 159:253 160:253 161:223 185:4 186:170 187:253 188:253 189:214 213:131 214:253 215:253 216:217 217:39 241:209 242:253 243:253 244:134 268:75 269:240 270:253 271:239 272:26 296:184 297:253 298:245 299:63 323:142 324:255 325:253 326:185 350:62 351:229 352:254 353:242 354:73 377:54 378:229 379:253 380:254 381:105 405:152 406:254 407:254 408:213 409:26 432:32 433:243 434:253 435:253 436:115 459:2 460:142 461:253 462:253 463:155 487:30 488:253 489:253 490:232 491:55 515:75 516:253 517:253 518:164 542:72 543:232 544:253 545:189 546:17 570:224 571:253 572:253 573:163 597:43 598:235 599:253 600:253 601:195 602:21 625:28 626:231 627:253 628:253 629:184 630:14 654:225 655:253 656:253 657:75 +0 155:21 156:176 157:253 158:253 159:124 182:105 183:176 184:251 185:251 186:251 187:251 188:105 208:58 209:217 210:241 211:253 212:251 213:251 214:251 215:251 216:243 217:113 218:5 235:63 236:231 237:251 238:251 239:253 240:251 241:251 242:251 243:251 244:253 245:251 246:113 263:144 264:251 265:251 266:251 267:253 268:251 269:251 270:251 271:251 272:253 273:251 274:215 290:125 291:253 292:253 293:253 294:253 295:255 296:253 297:253 298:253 299:253 300:255 301:253 302:227 303:42 318:253 319:251 320:251 321:251 322:251 323:253 324:251 325:251 326:251 327:251 328:253 329:251 330:251 331:142 345:27 346:253 347:251 348:251 349:235 350:241 351:253 352:251 353:246 354:137 355:35 356:98 357:251 358:251 359:236 360:61 372:47 373:211 374:253 375:251 376:235 377:82 378:103 379:253 380:251 381:137 384:73 385:251 386:251 387:251 388:71 399:27 400:211 401:251 402:253 403:251 404:86 407:72 408:71 409:10 412:73 413:251 414:251 415:173 416:20 427:89 428:253 429:253 430:255 431:253 432:35 440:73 441:253 442:253 443:253 444:72 454:84 455:236 456:251 457:251 458:253 459:251 460:138 468:73 469:251 470:251 471:251 472:71 481:63 482:236 483:251 484:251 485:251 486:227 487:251 488:246 489:138 490:11 494:16 495:37 496:228 497:251 498:246 499:137 500:10 509:73 510:251 511:251 512:251 513:173 514:42 515:142 516:142 517:142 518:41 522:109 523:251 524:253 525:251 526:137 537:73 538:251 539:251 540:173 541:20 549:27 550:211 551:251 552:253 553:147 554:10 565:73 566:253 567:253 568:143 575:21 576:176 577:253 578:253 579:253 593:73 594:251 595:251 596:205 597:144 603:176 604:251 605:251 606:188 607:107 621:62 622:236 623:251 624:251 625:251 626:218 627:217 628:217 629:217 630:217 631:253 632:230 633:189 634:20 650:83 651:158 652:251 653:251 654:253 655:251 656:251 657:251 658:251 659:253 660:107 679:37 680:251 681:251 682:253 683:251 684:251 685:251 686:122 687:72 688:30 +1 151:68 152:45 153:131 154:131 155:131 156:101 157:68 158:92 159:44 187:19 188:170 211:29 212:112 213:89 215:40 216:222 239:120 240:254 241:251 242:127 243:40 244:222 267:197 268:254 269:254 270:91 271:40 272:222 294:64 295:247 296:254 297:236 298:50 299:40 300:107 322:184 323:254 324:254 325:91 327:6 328:14 350:203 351:254 352:254 353:71 377:23 378:218 379:254 380:254 381:71 405:113 406:254 407:255 408:239 409:53 433:210 434:254 435:254 436:195 460:62 461:242 462:254 463:241 464:88 468:28 488:86 489:254 490:254 491:189 495:28 496:104 516:106 517:254 518:254 519:168 523:40 524:91 544:216 545:254 546:245 547:51 551:35 552:80 572:216 573:254 574:102 599:55 600:239 601:254 602:52 627:166 628:254 629:210 630:23 655:223 656:252 657:104 683:223 684:169 +0 125:29 126:170 127:255 128:255 129:141 151:29 152:198 153:255 154:255 155:255 156:226 157:255 158:86 178:141 179:255 180:255 181:170 182:29 184:86 185:255 186:255 187:141 204:29 205:226 206:255 207:198 208:57 213:226 214:255 215:255 216:226 217:114 231:29 232:255 233:255 234:114 241:141 242:170 243:114 244:255 245:255 246:141 259:226 260:255 261:170 269:29 270:57 273:141 274:255 275:226 286:57 287:255 288:170 302:114 303:255 304:198 314:226 315:255 331:170 332:255 333:57 342:255 343:226 360:255 361:170 370:255 371:170 388:114 389:198 398:255 399:226 416:86 417:255 426:198 427:255 444:86 445:255 454:114 455:255 456:57 472:86 473:255 482:29 483:255 484:226 500:141 501:255 511:170 512:255 513:170 528:226 529:198 539:29 540:226 541:255 542:170 555:29 556:255 557:114 568:29 569:226 570:255 571:141 582:57 583:226 584:226 598:141 599:255 600:255 601:170 602:86 607:29 608:86 609:226 610:255 611:226 612:29 627:86 628:198 629:255 630:255 631:255 632:255 633:255 634:255 635:255 636:255 637:255 638:141 639:29 657:29 658:114 659:170 660:170 661:170 662:170 663:170 664:86 +0 153:203 154:254 155:252 156:252 157:252 158:214 159:51 160:20 180:62 181:221 182:252 183:250 184:250 185:250 186:252 187:250 188:160 189:20 207:62 208:211 209:250 210:252 211:250 212:250 213:250 214:252 215:250 216:250 217:49 234:41 235:221 236:250 237:250 238:252 239:250 240:250 241:250 242:252 243:250 244:128 245:10 262:254 263:252 264:252 265:252 266:254 267:252 268:252 269:252 270:254 271:252 272:252 273:90 290:150 291:190 292:250 293:250 294:252 295:250 296:250 297:169 298:171 299:250 300:250 301:250 302:82 318:31 319:191 320:250 321:250 322:252 323:189 324:100 325:20 326:172 327:250 328:250 329:250 330:80 346:213 347:250 348:250 349:250 350:212 351:29 354:252 355:250 356:250 357:250 374:92 375:252 376:252 377:252 382:51 383:252 384:252 385:252 386:203 401:82 402:252 403:250 404:250 405:169 410:132 411:250 412:250 413:250 414:121 428:92 429:231 430:252 431:250 432:159 433:20 438:252 439:250 440:250 441:250 456:30 457:211 458:252 459:250 460:221 461:40 466:90 467:250 468:250 469:250 470:163 484:31 485:213 486:254 487:232 488:80 494:92 495:252 496:252 497:212 498:163 512:151 513:250 514:252 515:149 522:252 523:250 524:250 525:49 540:60 541:221 542:252 543:210 544:60 550:252 551:250 552:250 553:49 569:202 570:252 571:250 572:221 573:40 576:123 577:202 578:252 579:250 580:250 581:49 596:123 597:243 598:255 599:252 600:252 601:252 602:254 603:252 604:252 605:252 606:254 607:252 608:100 625:121 626:171 627:250 628:250 629:250 630:252 631:250 632:250 633:250 634:252 635:250 636:100 654:20 655:160 656:250 657:250 658:252 659:250 660:250 661:250 662:252 663:189 664:40 683:20 684:170 685:250 686:252 687:250 688:128 689:49 690:49 691:29 +1 98:64 99:191 100:70 125:68 126:243 127:253 128:249 129:63 152:30 153:223 154:253 155:253 156:247 157:41 179:73 180:238 181:253 182:253 183:253 184:242 206:73 207:236 208:253 209:253 210:253 211:253 212:242 234:182 235:253 236:253 237:191 238:247 239:253 240:149 262:141 263:253 264:143 265:86 266:249 267:253 268:122 290:9 291:36 292:7 293:14 294:233 295:253 296:122 322:230 323:253 324:122 350:230 351:253 352:122 378:231 379:255 380:123 406:230 407:253 408:52 433:61 434:245 435:253 461:98 462:253 463:253 468:35 469:12 489:98 490:253 491:253 494:9 495:142 496:233 497:146 517:190 518:253 519:253 520:128 521:7 522:99 523:253 524:253 525:180 544:29 545:230 546:253 547:253 548:252 549:210 550:253 551:253 552:253 553:140 571:28 572:207 573:253 574:253 575:253 576:254 577:253 578:253 579:235 580:70 581:9 599:126 600:253 601:253 602:253 603:253 604:254 605:253 606:168 607:19 627:79 628:253 629:253 630:201 631:190 632:132 633:63 634:5 +1 125:26 126:240 127:72 153:25 154:238 155:208 182:209 183:226 184:14 210:209 211:254 212:43 238:175 239:254 240:128 266:63 267:254 268:204 294:107 295:254 296:204 322:88 323:254 324:204 350:55 351:254 352:204 378:126 379:254 380:204 406:126 407:254 408:189 434:169 435:254 436:121 462:209 463:254 464:193 490:209 491:254 492:111 517:22 518:235 519:254 520:37 545:137 546:254 547:227 548:16 573:205 574:255 575:185 601:205 602:254 603:125 629:205 630:254 631:125 657:111 658:212 659:43 +0 155:62 156:91 157:213 158:255 159:228 160:91 161:12 182:70 183:230 184:253 185:253 186:253 187:253 188:253 189:152 190:7 210:246 211:253 212:253 213:253 214:253 215:253 216:253 217:253 218:106 237:21 238:247 239:253 240:253 241:253 242:253 243:253 244:253 245:208 246:24 265:156 266:253 267:253 268:253 269:253 270:253 271:253 272:253 273:195 292:88 293:238 294:253 295:253 296:253 297:221 298:253 299:253 300:253 301:195 320:230 321:253 322:253 323:253 324:198 325:40 326:177 327:253 328:253 329:195 346:56 347:156 348:251 349:253 350:189 351:182 352:15 354:86 355:240 356:253 357:210 358:28 374:213 375:253 376:253 377:156 378:3 383:205 384:253 385:253 386:106 401:121 402:252 403:253 404:135 405:3 411:46 412:253 413:253 414:106 428:28 429:212 430:253 431:248 432:23 439:42 440:253 441:253 442:106 456:197 457:253 458:234 459:70 467:42 468:253 469:253 470:106 483:11 484:202 485:253 486:187 495:58 496:253 497:210 498:27 511:107 512:253 513:253 514:40 522:53 523:227 524:253 525:195 539:107 540:253 541:253 542:40 549:47 550:227 551:253 552:231 553:58 567:107 568:253 569:253 570:40 575:5 576:131 577:222 578:253 579:231 580:59 595:14 596:204 597:253 598:226 599:222 600:73 601:58 602:58 603:170 604:253 605:253 606:227 607:58 624:197 625:253 626:253 627:253 628:253 629:253 630:253 631:253 632:253 633:238 634:58 652:33 653:179 654:241 655:253 656:253 657:253 658:253 659:250 660:116 661:14 682:75 683:179 684:253 685:151 686:89 687:86 +1 157:42 158:228 159:253 160:253 185:144 186:251 187:251 188:251 212:89 213:236 214:251 215:235 216:215 239:79 240:253 241:251 242:251 243:142 267:180 268:253 269:251 270:251 271:142 294:32 295:202 296:255 297:253 298:216 322:109 323:251 324:253 325:251 326:112 349:6 350:129 351:251 352:253 353:127 354:5 377:37 378:251 379:251 380:253 381:107 405:166 406:251 407:251 408:201 409:30 432:42 433:228 434:253 435:253 460:144 461:251 462:251 463:147 487:63 488:236 489:251 490:251 491:71 515:150 516:251 517:251 518:204 519:41 543:253 544:251 545:251 546:142 571:255 572:253 573:164 598:105 599:253 600:251 601:35 626:180 627:253 628:251 629:35 654:180 655:253 656:251 657:35 682:180 683:253 684:251 685:35 +1 128:62 129:254 130:213 156:102 157:253 158:252 159:102 160:20 184:102 185:254 186:253 187:254 188:50 212:102 213:253 214:252 215:253 216:50 240:102 241:254 242:253 243:254 244:50 268:142 269:253 270:252 271:253 272:50 295:51 296:253 297:254 298:253 299:224 300:20 323:132 324:252 325:253 326:252 327:162 351:173 352:253 353:254 354:253 355:102 378:82 379:253 380:252 381:253 382:252 383:61 406:203 407:254 408:253 409:254 410:233 433:41 434:243 435:253 436:252 437:253 438:111 461:132 462:253 463:254 464:253 465:203 488:41 489:253 490:252 491:253 492:252 493:40 515:11 516:213 517:254 518:253 519:254 520:151 543:92 544:252 545:253 546:252 547:192 548:50 570:21 571:214 572:253 573:255 574:253 575:41 598:142 599:253 600:252 601:253 602:171 625:113 626:253 627:255 628:253 629:203 630:40 653:30 654:131 655:233 656:111 +0 154:28 155:195 156:254 157:254 158:254 159:254 160:254 161:255 162:61 181:6 182:191 183:253 184:253 185:253 186:253 187:253 188:253 189:253 190:60 208:26 209:190 210:253 211:253 212:253 213:253 214:240 215:191 216:242 217:253 218:60 235:15 236:187 237:253 238:253 239:253 240:253 241:253 242:200 244:211 245:253 246:60 262:22 263:66 264:253 265:253 266:253 267:253 268:241 269:209 270:44 271:23 272:218 273:253 274:60 290:124 291:253 292:253 293:253 294:253 295:253 296:182 299:131 300:253 301:253 302:60 318:38 319:217 320:253 321:253 322:244 323:111 324:37 327:131 328:253 329:253 330:60 346:124 347:253 348:253 349:253 350:165 354:22 355:182 356:253 357:253 358:60 374:124 375:253 376:253 377:240 378:45 382:53 383:253 384:253 385:249 386:58 401:16 402:168 403:253 404:216 405:45 410:53 411:253 412:253 413:138 429:159 430:253 431:253 432:147 438:53 439:253 440:253 441:138 456:136 457:252 458:253 459:227 460:5 466:53 467:253 468:243 469:101 484:140 485:253 486:253 487:124 494:156 495:253 496:218 511:13 512:164 513:253 514:142 515:5 521:32 522:233 523:253 524:218 539:62 540:253 541:253 542:130 548:37 549:203 550:253 551:253 552:127 567:62 568:253 569:253 570:147 571:36 572:36 573:36 574:36 575:151 576:222 577:253 578:245 579:127 580:8 595:34 596:202 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:253 605:253 606:200 624:140 625:253 626:253 627:253 628:253 629:253 630:253 631:253 632:248 633:235 634:65 652:87 653:173 654:253 655:253 656:253 657:253 658:253 659:253 660:182 681:14 682:78 683:96 684:253 685:253 686:253 687:137 688:56 +0 123:8 124:76 125:202 126:254 127:255 128:163 129:37 130:2 150:13 151:182 152:253 153:253 154:253 155:253 156:253 157:253 158:23 177:15 178:179 179:253 180:253 181:212 182:91 183:218 184:253 185:253 186:179 187:109 205:105 206:253 207:253 208:160 209:35 210:156 211:253 212:253 213:253 214:253 215:250 216:113 232:19 233:212 234:253 235:253 236:88 237:121 238:253 239:233 240:128 241:91 242:245 243:253 244:248 245:114 260:104 261:253 262:253 263:110 264:2 265:142 266:253 267:90 270:26 271:199 272:253 273:248 274:63 287:1 288:173 289:253 290:253 291:29 293:84 294:228 295:39 299:72 300:251 301:253 302:215 303:29 315:36 316:253 317:253 318:203 319:13 328:82 329:253 330:253 331:170 343:36 344:253 345:253 346:164 356:11 357:198 358:253 359:184 360:6 371:36 372:253 373:253 374:82 385:138 386:253 387:253 388:35 399:128 400:253 401:253 402:47 413:48 414:253 415:253 416:35 427:154 428:253 429:253 430:47 441:48 442:253 443:253 444:35 455:102 456:253 457:253 458:99 469:48 470:253 471:253 472:35 483:36 484:253 485:253 486:164 496:16 497:208 498:253 499:211 500:17 511:32 512:244 513:253 514:175 515:4 524:44 525:253 526:253 527:156 540:171 541:253 542:253 543:29 551:30 552:217 553:253 554:188 555:19 568:171 569:253 570:253 571:59 578:60 579:217 580:253 581:253 582:70 596:78 597:253 598:253 599:231 600:48 604:26 605:128 606:249 607:253 608:244 609:94 610:15 624:8 625:151 626:253 627:253 628:234 629:101 630:121 631:219 632:229 633:253 634:253 635:201 636:80 653:38 654:232 655:253 656:253 657:253 658:253 659:253 660:253 661:253 662:201 663:66 +0 127:68 128:254 129:255 130:254 131:107 153:11 154:176 155:230 156:253 157:253 158:253 159:212 180:28 181:197 182:253 183:253 184:253 185:253 186:253 187:229 188:107 189:14 208:194 209:253 210:253 211:253 212:253 213:253 214:253 215:253 216:253 217:53 235:69 236:241 237:253 238:253 239:253 240:253 241:241 242:186 243:253 244:253 245:195 262:10 263:161 264:253 265:253 266:253 267:246 268:40 269:57 270:231 271:253 272:253 273:195 290:140 291:253 292:253 293:253 294:253 295:154 297:25 298:253 299:253 300:253 301:195 318:213 319:253 320:253 321:253 322:135 323:8 325:3 326:128 327:253 328:253 329:195 345:77 346:238 347:253 348:253 349:253 350:7 354:116 355:253 356:253 357:195 372:11 373:165 374:253 375:253 376:231 377:70 378:1 382:78 383:237 384:253 385:195 400:33 401:253 402:253 403:253 404:182 411:200 412:253 413:195 428:98 429:253 430:253 431:253 432:24 439:42 440:253 441:195 456:197 457:253 458:253 459:253 460:24 467:163 468:253 469:195 484:197 485:253 486:253 487:189 488:13 494:53 495:227 496:253 497:121 512:197 513:253 514:253 515:114 521:21 522:227 523:253 524:231 525:27 540:197 541:253 542:253 543:114 547:5 548:131 549:143 550:253 551:231 552:59 568:197 569:253 570:253 571:236 572:73 573:58 574:217 575:223 576:253 577:253 578:253 579:174 596:197 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:253 605:253 606:253 607:48 624:149 625:253 626:253 627:253 628:253 629:253 630:253 631:253 632:253 633:182 634:15 635:3 652:12 653:168 654:253 655:253 656:253 657:253 658:253 659:248 660:89 661:23 +1 157:85 158:255 159:103 160:1 185:205 186:253 187:253 188:30 213:205 214:253 215:253 216:30 240:44 241:233 242:253 243:244 244:27 268:135 269:253 270:253 271:100 296:153 297:253 298:240 299:76 323:12 324:208 325:253 326:166 351:69 352:253 353:253 354:142 378:14 379:110 380:253 381:235 382:33 406:63 407:223 408:235 409:130 434:186 435:253 436:235 437:37 461:17 462:145 463:253 464:231 465:35 489:69 490:220 491:231 492:123 516:18 517:205 518:253 519:176 520:27 543:17 544:125 545:253 546:185 547:39 571:71 572:214 573:231 574:41 599:167 600:253 601:225 602:33 626:72 627:205 628:207 629:14 653:30 654:249 655:233 656:49 681:32 682:253 683:89 +1 126:94 127:132 154:250 155:250 156:4 182:250 183:254 184:95 210:250 211:254 212:95 238:250 239:254 240:95 266:250 267:254 268:95 294:250 295:254 296:95 322:250 323:254 324:95 350:250 351:254 352:95 378:250 379:254 380:95 405:77 406:254 407:250 408:19 433:96 434:254 435:249 461:53 462:253 463:252 464:43 490:250 491:251 492:32 517:85 518:254 519:249 545:96 546:254 547:249 573:83 574:254 575:250 576:14 602:250 603:254 604:95 630:250 631:255 632:95 658:132 659:254 660:95 +1 124:32 125:253 126:31 152:32 153:251 154:149 180:32 181:251 182:188 208:32 209:251 210:188 236:32 237:251 238:228 239:59 264:32 265:253 266:253 267:95 292:28 293:236 294:251 295:114 321:127 322:251 323:251 349:127 350:251 351:251 377:48 378:232 379:251 406:223 407:253 408:159 434:221 435:251 436:158 462:142 463:251 464:158 490:64 491:251 492:242 493:55 518:64 519:251 520:253 521:161 546:64 547:253 548:255 549:221 574:16 575:181 576:253 577:220 603:79 604:253 605:236 606:63 632:213 633:251 634:126 660:96 661:251 662:126 +1 129:39 130:254 131:255 132:254 133:140 157:136 158:253 159:253 160:228 161:67 184:6 185:227 186:253 187:253 188:58 211:29 212:188 213:253 214:253 215:253 216:17 239:95 240:253 241:253 242:253 243:157 244:8 266:3 267:107 268:253 269:253 270:245 271:77 294:29 295:253 296:253 297:240 298:100 322:141 323:253 324:253 325:215 349:129 350:248 351:253 352:253 353:215 377:151 378:253 379:253 380:253 381:144 405:151 406:253 407:253 408:253 409:27 431:3 432:102 433:242 434:253 435:253 436:110 437:3 459:97 460:253 461:253 462:253 463:214 464:55 487:207 488:253 489:253 490:253 491:158 515:67 516:253 517:253 518:253 519:158 543:207 544:253 545:253 546:240 547:88 571:207 572:253 573:253 574:224 598:32 599:217 600:253 601:253 602:224 626:141 627:253 628:253 629:253 630:133 654:36 655:219 656:253 657:140 658:10 +0 123:59 124:55 149:71 150:192 151:254 152:250 153:147 154:17 176:123 177:247 178:253 179:254 180:253 181:253 182:196 183:79 184:176 185:175 186:175 187:124 188:48 203:87 204:247 205:247 206:176 207:95 208:102 209:117 210:243 211:237 212:192 213:232 214:253 215:253 216:245 217:152 218:6 230:23 231:229 232:253 233:138 238:219 239:58 241:95 242:118 243:80 244:230 245:254 246:196 247:30 258:120 259:254 260:205 261:8 266:114 272:38 273:255 274:254 275:155 276:5 286:156 287:253 288:92 301:61 302:235 303:253 304:102 314:224 315:253 316:78 330:117 331:253 332:196 333:18 342:254 343:253 344:78 358:9 359:211 360:253 361:73 370:254 371:253 372:78 387:175 388:253 389:155 398:194 399:254 400:101 415:79 416:254 417:155 426:112 427:253 428:211 429:9 443:73 444:251 445:200 454:41 455:241 456:253 457:87 471:25 472:240 473:253 483:147 484:253 485:227 486:47 499:94 500:253 501:200 511:5 512:193 513:253 514:230 515:76 527:175 528:253 529:155 540:31 541:219 542:254 543:255 544:126 545:18 553:14 554:149 555:254 556:244 557:45 569:21 570:158 571:254 572:253 573:226 574:162 575:118 576:96 577:20 578:20 579:73 580:118 581:224 582:253 583:247 584:85 598:30 599:155 600:253 601:253 602:253 603:253 604:254 605:253 606:253 607:253 608:253 609:254 610:247 611:84 627:5 628:27 629:117 630:206 631:244 632:229 633:213 634:213 635:213 636:176 637:117 638:32 659:45 660:23 +1 128:58 129:139 156:247 157:247 158:25 183:121 184:253 185:156 186:3 211:133 212:253 213:145 238:11 239:227 240:253 241:145 266:7 267:189 268:253 269:145 294:35 295:252 296:253 297:145 322:146 323:252 324:253 325:131 350:146 351:252 352:253 353:13 378:146 379:252 380:253 381:13 406:147 407:253 408:255 409:13 434:146 435:252 436:253 437:13 462:146 463:252 464:253 465:13 490:146 491:252 492:253 493:13 517:22 518:230 519:252 520:221 521:9 545:22 546:230 547:252 548:133 574:146 575:252 576:133 602:146 603:252 604:120 630:146 631:252 658:146 659:252 +1 129:28 130:247 131:255 132:165 156:47 157:221 158:252 159:252 160:164 184:177 185:252 186:252 187:252 188:164 212:177 213:252 214:252 215:223 216:78 240:177 241:252 242:252 243:197 267:114 268:236 269:252 270:235 271:42 294:5 295:148 296:252 297:252 298:230 321:14 322:135 323:252 324:252 325:252 326:230 349:78 350:252 351:252 352:252 353:252 354:162 377:78 378:252 379:252 380:252 381:252 382:9 405:78 406:252 407:252 408:252 409:252 410:9 432:32 433:200 434:252 435:252 436:252 437:105 438:3 459:10 460:218 461:252 462:252 463:252 464:105 465:8 487:225 488:252 489:252 490:252 491:240 492:69 514:44 515:237 516:252 517:252 518:228 519:85 541:59 542:218 543:252 544:252 545:225 546:93 568:65 569:208 570:252 571:252 572:252 573:175 596:133 597:252 598:252 599:252 600:225 601:68 624:133 625:252 626:252 627:244 628:54 652:133 653:252 654:252 655:48 +0 156:13 157:6 181:10 182:77 183:145 184:253 185:190 186:67 207:11 208:77 209:193 210:252 211:252 212:253 213:252 214:238 215:157 216:71 217:26 233:10 234:78 235:193 236:252 237:252 238:252 239:252 240:253 241:252 242:252 243:252 244:252 245:228 246:128 247:49 248:5 259:6 260:78 261:194 262:252 263:252 264:252 265:252 266:252 267:252 268:253 269:217 270:192 271:232 272:252 273:252 274:252 275:252 276:135 277:3 286:4 287:147 288:252 289:252 290:252 291:252 292:252 293:252 294:252 295:252 296:175 297:26 299:40 300:145 301:235 302:252 303:252 304:252 305:104 314:208 315:252 316:252 317:252 318:252 319:252 320:252 321:133 322:48 323:48 329:71 330:236 331:252 332:252 333:230 342:253 343:185 344:170 345:252 346:252 347:252 348:173 349:22 358:102 359:252 360:252 361:252 370:24 371:141 372:243 373:252 374:252 375:186 376:5 386:8 387:220 388:252 389:252 398:70 399:247 400:252 401:252 402:165 403:37 414:81 415:251 416:252 417:194 426:255 427:253 428:253 429:251 430:69 441:39 442:231 443:253 444:253 445:127 454:253 455:252 456:249 457:127 468:6 469:147 470:252 471:252 472:190 473:5 482:253 483:252 484:216 495:7 496:145 497:252 498:252 499:252 500:69 510:253 511:252 512:223 513:16 522:25 523:185 524:252 525:252 526:252 527:107 528:8 538:167 539:252 540:252 541:181 542:18 549:105 550:191 551:252 552:252 553:235 554:151 555:10 566:37 567:221 568:252 569:252 570:210 571:193 572:96 573:73 574:130 575:188 576:194 577:227 578:252 579:252 580:235 581:128 595:97 596:220 597:252 598:252 599:252 600:252 601:252 602:252 603:252 604:253 605:252 606:252 607:236 608:70 624:40 625:174 626:252 627:252 628:252 629:252 630:252 631:252 632:253 633:197 634:138 635:29 653:5 654:23 655:116 656:143 657:143 658:143 659:143 660:24 661:10 +0 127:28 128:164 129:254 130:233 131:148 132:11 154:3 155:164 156:254 157:234 158:225 159:254 160:204 182:91 183:254 184:235 185:48 186:32 187:166 188:251 189:92 208:33 209:111 210:214 211:205 212:49 215:24 216:216 217:210 235:34 236:217 237:254 238:254 239:211 244:87 245:237 246:43 262:34 263:216 264:254 265:254 266:252 267:243 268:61 272:38 273:248 274:182 290:171 291:254 292:184 293:205 294:175 295:36 301:171 302:227 317:28 318:234 319:190 320:13 321:193 322:157 329:124 330:238 331:26 345:140 346:254 347:131 349:129 350:157 357:124 358:254 359:95 373:201 374:238 375:56 377:70 378:103 385:124 386:254 387:148 400:62 401:255 402:210 413:150 414:254 415:122 428:86 429:254 430:201 431:15 440:28 441:237 442:246 443:44 456:128 457:254 458:143 468:34 469:243 470:227 484:62 485:254 486:210 496:58 497:249 498:179 512:30 513:240 514:210 524:207 525:254 526:64 541:216 542:231 543:34 551:129 552:248 553:170 554:9 569:131 570:254 571:170 577:17 578:129 579:248 580:225 581:24 597:50 598:245 599:245 600:184 601:106 602:106 603:106 604:133 605:231 606:254 607:244 608:53 626:67 627:249 628:254 629:254 630:254 631:254 632:254 633:251 634:193 635:40 655:38 656:157 657:248 658:166 659:166 660:139 661:57 +0 129:105 130:255 131:219 132:67 133:67 134:52 156:20 157:181 158:253 159:253 160:253 161:253 162:226 163:69 182:4 183:129 184:206 185:253 186:253 187:253 188:253 189:253 190:253 191:130 209:9 210:141 211:253 212:253 213:253 214:253 215:253 216:253 217:253 218:253 219:166 220:20 237:134 238:253 239:253 240:253 241:253 242:253 243:253 244:253 245:253 246:253 247:253 248:65 262:2 263:83 264:207 265:246 266:253 267:253 268:253 269:253 270:253 271:249 272:234 273:247 274:253 275:253 276:65 290:83 291:253 292:253 293:253 294:253 295:253 296:189 297:253 298:253 299:205 301:179 302:253 303:253 304:65 317:85 318:234 319:253 320:253 321:253 322:253 323:157 324:26 325:164 326:151 327:83 329:179 330:253 331:253 332:65 344:65 345:237 346:253 347:253 348:253 349:67 350:36 351:14 353:15 354:12 357:179 358:253 359:253 360:65 371:4 372:141 373:253 374:253 375:221 376:158 377:23 385:179 386:253 387:253 388:65 399:129 400:253 401:253 402:241 403:62 412:72 413:226 414:253 415:175 416:24 426:119 427:247 428:253 429:253 430:206 439:8 440:134 441:253 442:253 443:130 454:132 455:253 456:253 457:194 458:27 467:125 468:253 469:253 470:253 471:130 481:45 482:213 483:253 484:253 485:112 493:70 494:170 495:247 496:253 497:253 498:89 499:43 509:67 510:253 511:253 512:196 513:55 514:9 520:8 521:131 522:253 523:253 524:253 525:86 526:1 537:67 538:253 539:253 540:253 541:253 542:129 546:43 547:114 548:134 549:253 550:253 551:231 552:139 553:41 565:20 566:167 567:253 568:253 569:253 570:247 571:179 572:179 573:179 574:206 575:253 576:253 577:253 578:253 579:72 594:103 595:240 596:253 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:244 605:119 606:8 607:1 623:107 624:253 625:253 626:253 627:253 628:253 629:253 630:253 631:175 632:111 651:3 652:121 653:253 654:253 655:253 656:253 657:253 658:182 659:24 +0 125:22 126:183 127:252 128:254 129:252 130:252 131:252 132:76 151:85 152:85 153:168 154:250 155:250 156:252 157:250 158:250 159:250 160:250 161:71 163:43 164:85 165:14 178:107 179:252 180:250 181:250 182:250 183:250 184:252 185:250 186:250 187:250 188:250 189:210 191:127 192:250 193:146 205:114 206:237 207:252 208:250 209:250 210:250 211:250 212:252 213:250 214:250 215:250 216:250 217:210 219:127 220:250 221:250 232:107 233:237 234:250 235:252 236:250 237:250 238:250 239:74 240:41 241:41 242:41 243:41 244:217 245:34 247:127 248:250 249:250 259:15 260:148 261:252 262:252 263:254 264:238 265:105 275:128 276:252 277:252 286:15 287:140 288:250 289:250 290:250 291:167 292:111 303:127 304:250 305:250 314:43 315:250 316:250 317:250 318:250 331:127 332:250 333:250 342:183 343:250 344:250 345:250 346:110 358:57 359:210 360:250 361:250 370:252 371:250 372:250 373:110 374:7 386:85 387:250 388:250 389:250 398:254 399:252 400:252 401:83 414:86 415:252 416:252 417:217 426:252 427:250 428:250 429:138 430:14 441:15 442:140 443:250 444:250 445:41 454:252 455:250 456:250 457:250 458:41 469:43 470:250 471:250 472:250 473:41 482:252 483:250 484:250 485:250 486:181 497:183 498:250 499:250 500:250 501:41 510:76 511:250 512:250 513:250 514:250 524:177 525:252 526:250 527:250 528:110 529:7 538:36 539:224 540:252 541:252 542:252 543:219 544:43 545:43 546:43 547:7 549:15 550:43 551:183 552:252 553:255 554:252 555:126 567:85 568:250 569:250 570:250 571:252 572:250 573:250 574:250 575:111 576:86 577:140 578:250 579:250 580:250 581:252 582:222 583:83 595:42 596:188 597:250 598:250 599:252 600:250 601:250 602:250 603:250 604:252 605:250 606:250 607:250 608:250 609:126 610:83 624:127 625:250 626:250 627:252 628:250 629:250 630:250 631:250 632:252 633:250 634:250 635:137 636:83 652:21 653:41 654:217 655:252 656:250 657:250 658:250 659:250 660:217 661:41 662:41 663:14 +1 155:114 156:206 157:25 183:238 184:252 185:55 211:222 212:252 213:55 239:113 240:252 241:55 267:113 268:252 269:55 295:255 296:253 297:56 323:253 324:176 325:6 350:32 351:253 352:233 353:43 378:140 379:253 380:195 381:19 406:140 407:253 408:167 433:29 434:253 435:141 461:29 462:252 463:140 489:29 490:252 491:140 517:29 518:252 519:140 545:29 546:252 547:140 573:169 574:253 575:79 601:169 602:252 628:76 629:234 630:141 656:197 657:233 658:37 684:197 685:223 +1 127:73 128:253 129:253 130:63 155:115 156:252 157:252 158:144 183:217 184:252 185:252 186:144 210:63 211:237 212:252 213:252 214:144 238:109 239:252 240:252 241:252 266:109 267:252 268:252 269:252 294:109 295:252 296:252 297:252 322:191 323:252 324:252 325:252 349:145 350:255 351:253 352:253 353:253 376:32 377:237 378:253 379:252 380:252 381:210 404:37 405:252 406:253 407:252 408:252 409:108 432:37 433:252 434:253 435:252 436:252 437:108 460:21 461:207 462:255 463:253 464:253 465:108 489:144 490:253 491:252 492:252 493:108 516:27 517:221 518:253 519:252 520:252 521:108 544:16 545:190 546:253 547:252 548:252 549:108 573:145 574:255 575:253 576:253 577:253 601:144 602:253 603:252 604:252 605:210 629:144 630:253 631:252 632:252 633:108 657:62 658:253 659:252 660:252 661:108 +1 120:85 121:253 122:132 123:9 147:82 148:241 149:251 150:251 151:128 175:175 176:251 177:251 178:251 179:245 180:121 203:13 204:204 205:251 206:251 207:251 208:245 209:107 232:39 233:251 234:251 235:251 236:251 237:167 238:22 260:15 261:155 262:251 263:251 264:251 265:251 266:177 289:15 290:157 291:248 292:251 293:251 294:251 295:165 319:214 320:251 321:251 322:251 323:212 324:78 325:24 347:109 348:251 349:251 350:251 351:253 352:251 353:170 354:10 375:5 376:57 377:162 378:251 379:253 380:251 381:251 382:18 405:106 406:239 407:255 408:253 409:253 410:213 434:105 435:253 436:251 437:251 438:230 439:72 463:253 464:251 465:251 466:251 467:221 468:67 491:72 492:251 493:251 494:251 495:251 496:96 519:36 520:199 521:251 522:251 523:251 524:155 525:15 548:45 549:204 550:251 551:251 552:251 553:157 577:161 578:249 579:251 580:251 581:248 582:147 606:233 607:251 608:251 609:251 610:173 634:233 635:251 636:251 637:251 638:173 662:53 663:131 664:251 665:251 666:173 +1 126:15 127:200 128:255 129:90 154:42 155:254 156:254 157:173 182:42 183:254 184:254 185:199 210:26 211:237 212:254 213:221 214:12 239:213 240:254 241:231 242:17 267:213 268:254 269:199 295:213 296:254 297:199 323:213 324:254 325:96 350:20 351:232 352:254 353:33 378:84 379:254 380:229 381:17 406:168 407:254 408:203 433:8 434:217 435:254 436:187 461:84 462:254 463:254 464:48 489:195 490:254 491:254 492:37 516:20 517:233 518:254 519:212 520:4 544:132 545:254 546:254 547:82 571:9 572:215 573:254 574:254 575:116 576:46 599:55 600:254 601:254 602:254 603:254 604:121 627:113 628:254 629:254 630:254 631:254 632:40 655:12 656:163 657:254 658:185 659:58 660:1 +0 182:32 183:57 184:57 185:57 186:57 187:57 188:57 189:57 208:67 209:185 210:229 211:252 212:252 213:252 214:253 215:252 216:252 217:252 218:185 219:66 234:13 235:188 236:246 237:252 238:253 239:252 240:252 241:252 242:241 243:139 244:177 245:252 246:253 247:246 248:187 249:13 261:26 262:255 263:253 264:244 265:175 266:101 274:126 275:244 276:253 277:153 288:82 289:243 290:253 291:214 292:81 303:169 304:252 305:252 315:19 316:215 317:252 318:206 319:56 331:169 332:252 333:252 343:157 344:252 345:252 346:13 359:169 360:252 361:151 370:41 371:253 372:253 373:128 386:92 387:253 388:206 389:13 398:166 399:252 400:196 401:9 414:216 415:252 416:142 426:253 427:252 428:168 441:89 442:253 443:208 444:13 454:253 455:252 456:68 468:38 469:225 470:253 471:96 482:254 483:253 484:56 495:45 496:229 497:253 498:151 510:253 511:252 512:81 522:70 523:225 524:252 525:227 538:216 539:252 540:168 548:29 549:134 550:253 551:252 552:186 553:31 566:91 567:252 568:243 569:125 573:51 574:114 575:113 576:210 577:252 578:253 579:151 580:19 595:157 596:253 597:253 598:254 599:253 600:253 601:253 602:254 603:253 604:244 605:175 606:51 623:19 624:122 625:196 626:197 627:221 628:196 629:196 630:197 631:121 632:56 655:25 +0 127:42 128:235 129:255 130:84 153:15 154:132 155:208 156:253 157:253 158:171 159:108 180:6 181:177 182:253 183:253 184:253 185:253 186:253 187:242 188:110 208:151 209:253 210:253 211:253 212:253 213:253 214:253 215:253 216:139 235:48 236:208 237:253 238:253 239:253 240:253 241:253 242:253 243:253 244:139 263:85 264:253 265:253 266:253 267:253 268:236 269:156 270:184 271:253 272:148 273:6 290:7 291:141 292:253 293:253 294:253 295:253 296:27 298:170 299:253 300:253 301:74 318:19 319:253 320:253 321:253 322:253 323:253 324:27 326:170 327:253 328:253 329:74 345:16 346:186 347:253 348:253 349:253 350:242 351:105 352:4 354:170 355:253 356:253 357:94 358:1 373:141 374:253 375:253 376:253 377:242 378:100 382:170 383:253 384:253 385:253 386:8 401:141 402:253 403:253 404:253 405:224 410:170 411:253 412:253 413:253 414:8 428:12 429:158 430:253 431:253 432:230 433:51 438:18 439:237 440:253 441:253 442:8 456:76 457:253 458:253 459:218 460:61 467:236 468:253 469:253 470:8 484:76 485:253 486:253 487:168 495:110 496:253 497:132 498:3 512:76 513:253 514:253 515:168 521:20 522:174 523:239 524:147 525:5 539:5 540:155 541:253 542:253 543:168 548:102 549:170 550:253 551:253 552:139 567:3 568:128 569:253 570:253 571:228 572:179 573:179 574:179 575:179 576:245 577:253 578:253 579:219 580:41 596:76 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:253 605:253 606:253 607:163 624:39 625:199 626:253 627:253 628:253 629:253 630:253 631:253 632:253 633:253 634:170 635:9 653:36 654:219 655:253 656:253 657:253 658:253 659:253 660:224 661:65 662:22 +1 156:202 157:253 158:69 184:253 185:252 186:121 212:253 213:252 214:69 240:253 241:252 242:69 267:106 268:253 269:231 270:37 295:179 296:255 297:196 322:17 323:234 324:253 325:92 350:93 351:252 352:253 353:92 378:93 379:252 380:253 381:92 406:93 407:252 408:232 409:8 434:208 435:253 436:116 462:207 463:252 464:116 490:207 491:252 492:32 517:57 518:244 519:252 545:122 546:252 547:252 573:185 574:253 575:253 601:184 602:252 603:252 629:101 630:252 631:252 657:13 658:173 659:252 660:43 686:9 687:232 688:116 +1 156:73 157:253 158:253 159:253 160:124 184:73 185:251 186:251 187:251 188:251 212:99 213:251 214:251 215:251 216:225 240:253 241:251 242:251 243:251 244:71 266:79 267:180 268:253 269:251 270:251 271:173 272:20 294:110 295:253 296:255 297:253 298:216 322:109 323:251 324:253 325:251 326:215 350:109 351:251 352:253 353:251 354:215 378:109 379:251 380:253 381:251 382:137 406:109 407:251 408:253 409:251 410:35 433:37 434:253 435:253 436:255 437:253 438:35 461:140 462:251 463:251 464:253 465:168 466:15 488:125 489:246 490:251 491:251 492:190 493:15 516:144 517:251 518:251 519:251 520:180 543:53 544:221 545:251 546:251 547:251 548:51 571:125 572:253 573:253 574:253 575:201 598:105 599:253 600:251 601:251 602:188 603:30 626:180 627:253 628:251 629:251 630:142 654:180 655:253 656:251 657:235 658:82 682:180 683:253 684:251 685:215 +1 124:111 125:255 126:48 152:162 153:253 154:237 155:63 180:206 181:253 182:253 183:183 208:87 209:217 210:253 211:205 237:90 238:253 239:238 240:60 265:37 266:225 267:253 268:89 294:206 295:253 296:159 322:206 323:253 324:226 350:206 351:253 352:226 378:206 379:253 380:226 406:206 407:253 408:226 434:206 435:253 436:226 462:206 463:253 464:226 490:206 491:253 492:226 518:206 519:253 520:237 521:45 546:206 547:253 548:253 549:109 574:173 575:253 576:253 577:109 602:69 603:253 604:253 605:109 630:64 631:248 632:253 633:109 659:112 660:253 661:109 +0 99:70 100:255 101:165 102:114 127:122 128:253 129:253 130:253 131:120 155:165 156:253 157:253 158:253 159:234 160:52 183:99 184:253 185:253 186:253 187:253 188:228 189:26 209:60 210:168 211:238 212:202 213:174 214:253 215:253 216:253 217:127 235:91 236:81 237:1 238:215 239:128 240:28 241:12 242:181 243:253 244:253 245:175 246:3 262:18 263:204 264:253 265:77 270:7 271:253 272:253 273:253 274:54 289:54 290:248 291:253 292:253 293:143 298:1 299:127 300:253 301:253 302:188 317:104 318:253 319:253 320:253 321:20 327:81 328:249 329:253 330:191 345:192 346:253 347:253 348:218 349:5 356:203 357:253 358:208 359:21 372:56 373:237 374:253 375:250 376:100 384:104 385:253 386:253 387:75 400:76 401:253 402:253 403:224 412:119 413:253 414:253 415:75 428:80 429:253 430:253 431:103 439:4 440:241 441:253 442:218 443:32 456:213 457:253 458:253 459:103 467:125 468:253 469:253 470:191 484:213 485:253 486:253 487:103 494:3 495:176 496:253 497:253 498:135 512:213 513:253 514:253 515:103 521:9 522:162 523:253 524:253 525:226 526:37 540:179 541:253 542:253 543:135 548:46 549:157 550:253 551:253 552:253 553:63 568:23 569:188 570:253 571:249 572:179 573:179 574:179 575:179 576:233 577:253 578:253 579:233 580:156 581:10 597:51 598:235 599:253 600:253 601:253 602:253 603:253 604:253 605:251 606:232 607:120 626:16 627:124 628:253 629:253 630:253 631:253 632:152 633:104 +1 124:29 125:197 126:255 127:84 152:85 153:251 154:253 155:83 180:86 181:253 182:254 183:253 208:85 209:251 210:253 211:251 236:86 237:253 238:254 239:253 240:169 264:85 265:251 266:253 267:251 268:168 292:86 293:253 294:254 295:253 296:169 320:28 321:196 322:253 323:251 324:168 349:169 350:254 351:253 352:169 377:168 378:253 379:251 380:168 405:169 406:254 407:253 408:169 433:168 434:253 435:251 436:168 462:254 463:253 464:254 465:139 490:253 491:251 492:253 493:251 518:254 519:253 520:254 521:253 522:57 546:253 547:251 548:253 549:251 550:168 574:198 575:253 576:254 577:253 578:114 602:85 603:251 604:253 605:251 630:85 631:253 632:254 633:253 658:28 659:83 660:196 661:83 +1 159:31 160:210 161:253 162:163 187:198 188:252 189:252 190:162 213:10 214:86 215:242 216:252 217:252 218:66 241:164 242:252 243:252 244:252 245:188 246:8 268:53 269:242 270:252 271:252 272:225 273:14 296:78 297:252 298:252 299:252 300:204 323:56 324:231 325:252 326:252 327:212 328:35 351:157 352:252 353:252 354:252 355:37 377:8 378:132 379:253 380:252 381:252 382:230 383:24 405:45 406:252 407:253 408:252 409:154 410:55 427:7 428:55 433:107 434:253 435:255 436:228 437:53 454:15 455:24 456:23 460:110 461:242 462:252 463:228 464:59 482:57 483:83 487:88 488:247 489:252 490:252 491:140 514:15 515:189 516:252 517:252 518:252 542:74 543:252 544:252 545:238 546:90 570:178 571:252 572:252 573:189 597:40 598:217 599:252 600:252 601:59 625:75 626:252 627:252 628:252 629:85 630:61 653:62 654:239 655:252 656:156 657:14 682:178 683:252 684:14 +1 131:159 132:255 133:122 158:167 159:228 160:253 161:121 185:64 186:236 187:251 188:205 189:110 212:48 213:158 214:251 215:251 216:178 217:39 240:190 241:251 242:251 243:251 267:96 268:253 269:253 270:253 271:153 295:194 296:251 297:251 298:211 299:74 322:80 323:174 324:251 325:251 326:140 327:47 349:16 350:181 351:253 352:251 353:219 354:23 377:64 378:251 379:253 380:251 381:204 382:19 405:223 406:253 407:255 408:233 409:48 431:20 432:174 433:244 434:251 435:253 436:109 437:31 459:96 460:189 461:251 462:251 463:126 464:31 486:24 487:106 488:251 489:235 490:188 491:100 514:96 515:251 516:251 517:228 518:59 542:255 543:253 544:253 545:213 546:36 569:100 570:253 571:251 572:251 573:85 574:23 596:32 597:127 598:253 599:235 600:126 601:15 624:104 625:251 626:253 627:240 628:79 652:83 653:193 654:253 655:220 +0 153:92 154:191 155:178 156:253 157:242 158:141 159:104 160:29 180:26 181:253 182:252 183:252 184:252 185:253 186:252 187:252 188:252 189:108 190:19 206:57 207:123 208:222 209:253 210:252 211:252 212:252 213:168 214:224 215:252 216:252 217:253 218:84 233:176 234:243 235:252 236:252 237:253 238:252 239:252 240:252 242:19 243:153 244:252 245:253 246:209 247:25 259:10 260:128 261:255 262:253 263:244 264:225 265:114 266:194 267:253 268:178 272:163 273:254 274:253 275:168 287:85 288:252 289:253 290:189 291:56 294:19 295:133 296:9 300:38 301:253 302:252 303:168 314:19 315:191 316:252 317:194 318:19 329:253 330:252 331:234 332:22 342:107 343:252 344:252 345:13 357:253 358:252 359:252 360:128 370:169 371:253 372:241 385:141 386:253 387:253 388:140 397:19 398:225 399:252 400:139 413:66 414:252 415:252 416:139 425:29 426:252 427:252 428:52 441:29 442:252 443:252 444:139 453:29 454:252 455:252 456:28 469:29 470:252 471:252 472:40 481:141 482:253 483:253 484:91 497:154 498:253 499:168 509:66 510:252 511:252 512:165 525:253 526:252 527:168 537:19 538:224 539:252 540:252 552:126 553:253 554:252 555:80 566:169 567:252 568:252 569:214 570:38 579:126 580:249 581:253 582:151 583:6 594:26 595:223 596:253 597:254 598:253 599:128 600:29 604:13 605:41 606:216 607:253 608:253 609:226 610:38 623:122 624:252 625:253 626:252 627:252 628:252 629:169 630:169 631:169 632:206 633:253 634:252 635:252 636:202 637:38 651:19 652:56 653:168 654:224 655:252 656:252 657:253 658:252 659:252 660:252 661:253 662:233 663:130 664:6 682:94 683:139 684:190 685:153 686:252 687:164 688:139 689:28 690:22 +1 128:53 129:250 130:255 131:25 156:167 157:253 158:253 159:25 182:3 183:123 184:247 185:253 186:253 187:25 210:9 211:253 212:253 213:253 214:253 215:25 238:9 239:253 240:253 241:253 242:253 243:25 266:9 267:253 268:253 269:253 270:180 271:13 294:9 295:253 296:253 297:253 298:104 322:9 323:253 324:253 325:253 326:104 350:15 351:253 352:253 353:253 354:104 378:184 379:253 380:253 381:228 382:68 406:184 407:253 408:253 409:182 433:103 434:251 435:253 436:253 437:12 461:106 462:253 463:253 464:253 465:8 488:24 489:238 490:253 491:253 492:253 493:8 516:27 517:253 518:253 519:253 520:253 521:8 544:27 545:253 546:253 547:253 548:253 549:8 572:27 573:253 574:253 575:253 576:177 577:4 600:160 601:253 602:253 603:253 604:87 628:202 629:253 630:253 631:219 632:54 656:81 657:253 658:247 659:51 +0 122:63 123:176 124:253 125:253 126:159 127:113 128:63 150:140 151:253 152:252 153:252 154:252 155:252 156:241 157:100 158:66 177:54 178:227 179:253 180:252 181:252 182:252 183:252 184:253 185:252 186:239 187:181 188:57 204:38 205:224 206:252 207:253 208:226 209:246 210:252 211:252 212:253 213:252 214:252 215:252 216:252 217:108 218:3 232:57 233:252 234:252 235:253 236:27 237:88 238:112 239:112 240:112 241:112 242:142 243:252 244:252 245:253 246:152 247:31 260:198 261:253 262:253 263:79 270:32 271:153 272:253 273:255 274:253 275:196 287:76 288:246 289:252 290:127 299:3 300:106 301:253 302:252 303:214 304:28 315:194 316:252 317:252 318:112 329:143 330:252 331:252 332:193 343:225 344:252 345:217 346:37 357:38 358:234 359:252 360:223 370:63 371:240 372:252 373:84 386:146 387:252 388:223 398:114 399:253 400:228 401:47 414:147 415:253 416:253 417:112 426:159 427:252 428:195 442:225 443:252 444:252 445:112 454:253 455:252 456:195 470:225 471:252 472:230 473:25 482:159 483:252 484:202 485:10 497:92 498:243 499:252 500:208 510:113 511:252 512:252 513:161 524:79 525:253 526:252 527:220 528:37 538:114 539:253 540:253 541:253 542:174 543:63 550:26 551:128 552:253 553:255 554:253 555:133 566:12 567:228 568:252 569:252 570:252 571:241 572:100 573:85 574:76 576:85 577:131 578:231 579:252 580:252 581:253 582:129 583:6 595:97 596:208 597:252 598:252 599:253 600:252 601:252 602:246 603:197 604:253 605:252 606:252 607:252 608:220 609:133 610:6 624:19 625:99 626:239 627:253 628:252 629:252 630:252 631:252 632:253 633:252 634:245 635:223 636:99 654:63 655:112 656:112 657:221 658:252 659:252 660:253 661:127 662:87 +0 153:12 154:136 155:254 156:255 157:195 158:115 159:3 180:6 181:175 182:253 183:196 184:160 185:252 186:253 187:15 208:130 209:253 210:234 211:4 213:27 214:205 215:232 216:40 235:54 236:246 237:253 238:68 242:24 243:243 244:106 262:3 263:134 264:235 265:99 266:4 271:132 272:247 273:77 290:56 291:253 292:62 299:23 300:233 301:129 318:179 319:183 320:4 328:182 329:220 345:21 346:232 347:59 356:95 357:232 358:21 373:128 374:183 385:228 386:85 401:187 402:124 413:228 414:186 429:187 430:124 441:228 442:104 457:187 458:124 469:169 470:184 485:187 486:124 497:203 498:150 513:187 514:124 524:10 525:220 526:39 541:187 542:155 552:111 553:201 569:129 570:228 571:7 579:12 580:181 581:76 598:234 599:166 600:9 606:24 607:209 608:106 626:139 627:250 628:167 629:11 630:2 631:11 632:11 633:129 634:227 635:90 636:11 655:95 656:247 657:253 658:178 659:253 660:253 661:244 662:86 684:47 685:175 686:253 687:232 688:149 689:40 +1 128:255 129:253 130:57 156:253 157:251 158:225 159:56 183:169 184:254 185:253 186:254 187:84 211:168 212:253 213:251 214:253 215:83 238:85 239:253 240:254 241:253 242:169 266:85 267:251 268:253 269:251 270:56 294:141 295:253 296:254 297:253 322:253 323:251 324:253 325:251 350:254 351:253 352:254 353:253 378:253 379:251 380:253 381:251 406:254 407:253 408:254 409:196 433:114 434:253 435:251 436:253 437:83 461:169 462:254 463:253 464:226 465:56 489:168 490:253 491:251 492:168 516:85 517:253 518:254 519:253 544:85 545:251 546:253 547:251 572:254 573:253 574:254 575:253 600:253 601:251 602:253 603:251 628:254 629:253 630:254 631:253 656:139 657:251 658:253 659:138 +0 151:23 152:167 153:208 154:254 155:255 156:129 157:19 179:151 180:253 181:253 182:253 183:253 184:253 185:209 186:26 207:181 208:253 209:253 210:253 211:227 212:181 213:253 214:207 215:22 235:227 236:253 237:253 238:253 239:92 240:38 241:226 242:253 243:129 244:2 263:193 264:253 265:253 266:248 267:62 269:50 270:253 271:253 272:45 291:170 292:253 293:253 294:135 297:12 298:208 299:253 300:119 318:16 319:232 320:253 321:253 322:21 326:60 327:253 328:185 346:164 347:253 348:253 349:224 350:14 354:14 355:217 356:247 357:62 373:3 374:193 375:253 376:250 377:64 383:199 384:253 385:179 401:67 402:253 403:253 404:205 411:98 412:253 413:188 429:151 430:253 431:245 432:43 439:63 440:250 441:188 457:151 458:253 459:243 468:244 469:222 470:22 485:151 486:253 487:217 496:244 497:253 498:115 512:3 513:195 514:253 515:134 524:156 525:253 526:150 541:140 542:253 543:134 552:239 553:253 554:139 569:44 570:253 571:134 579:53 580:246 581:237 582:32 597:8 598:200 599:229 600:40 606:25 607:225 608:253 609:188 626:120 627:250 628:230 629:58 630:17 632:12 633:42 634:213 635:253 636:238 637:84 655:151 656:253 657:253 658:217 659:179 660:206 661:253 662:253 663:196 664:118 683:18 684:58 685:145 686:152 687:253 688:214 689:145 690:74 691:7 +1 130:24 131:150 132:233 133:38 156:14 157:89 158:253 159:254 160:254 161:71 183:78 184:203 185:254 186:254 187:254 188:232 189:77 190:54 191:8 209:12 210:155 211:240 212:254 213:223 214:76 215:254 216:254 217:254 218:254 219:68 235:3 236:101 237:216 238:254 239:227 240:122 241:26 242:110 243:254 244:254 245:254 246:184 247:100 262:46 263:222 264:254 265:254 266:179 267:48 270:181 271:254 272:254 273:146 274:6 288:2 289:145 290:248 291:254 292:182 293:111 294:4 297:3 298:250 299:254 300:206 301:3 315:6 316:144 317:254 318:254 319:171 325:125 326:254 327:252 328:80 342:6 343:142 344:254 345:179 346:95 347:4 352:61 353:246 354:254 355:150 370:64 371:254 372:177 373:14 380:124 381:254 382:246 383:32 398:108 399:97 400:15 407:24 408:226 409:254 410:116 435:177 436:255 437:254 438:5 463:196 464:254 465:99 466:1 490:3 491:199 492:254 493:79 518:129 519:254 520:254 521:23 546:178 547:254 548:192 549:8 550:3 551:43 573:11 574:198 575:254 576:128 577:66 578:130 579:225 595:137 596:202 597:106 598:84 599:84 600:84 601:112 602:254 603:254 604:254 605:254 606:212 607:151 623:172 624:254 625:254 626:254 627:254 628:254 629:254 630:254 631:254 632:254 633:162 634:75 651:12 652:106 653:177 654:254 655:254 656:254 657:235 658:135 659:100 660:17 661:2 +0 125:120 126:253 127:253 128:63 151:38 152:131 153:246 154:252 155:252 156:203 157:15 179:222 180:252 181:252 182:252 183:252 184:166 185:38 205:4 206:107 207:253 208:252 209:252 210:252 211:252 212:253 213:224 214:137 215:26 233:107 234:252 235:253 236:252 237:220 238:128 239:252 240:253 241:252 242:252 243:239 244:140 261:170 262:253 263:255 264:168 267:79 268:192 269:253 270:253 271:253 272:253 273:255 274:90 288:51 289:243 290:252 291:215 292:33 296:12 297:74 298:233 299:252 300:252 301:253 302:195 303:19 316:166 317:252 318:252 319:31 326:43 327:149 328:195 329:253 330:252 331:177 332:19 343:57 344:234 345:252 346:252 357:237 358:252 359:252 360:180 361:13 371:85 372:252 373:252 374:173 385:50 386:237 387:252 388:252 389:112 399:226 400:253 401:240 402:63 414:163 415:253 416:253 417:112 426:38 427:234 428:252 429:176 442:85 443:252 444:252 445:158 454:113 455:252 456:252 457:84 470:19 471:209 472:252 473:252 482:207 483:252 484:252 485:84 498:10 499:203 500:252 501:236 510:253 511:252 512:252 513:84 526:85 527:252 528:252 529:112 538:114 539:253 540:253 541:146 553:51 554:159 555:253 556:240 557:63 566:75 567:243 568:252 569:249 570:146 579:57 580:85 581:238 582:252 583:252 584:99 595:116 596:252 597:252 598:252 599:198 600:197 601:165 602:57 603:57 604:57 605:182 606:197 607:234 608:252 609:253 610:233 611:164 612:19 623:28 624:84 625:180 626:252 627:253 628:252 629:252 630:252 631:252 632:253 633:252 634:252 635:252 636:252 637:225 638:71 653:13 654:112 655:253 656:252 657:252 658:252 659:252 660:253 661:252 662:252 663:157 664:112 +1 127:155 128:253 129:126 155:253 156:251 157:141 158:4 183:253 184:251 185:251 186:31 211:253 212:251 213:251 214:31 239:253 240:251 241:251 242:31 267:255 268:253 269:253 270:31 293:8 294:131 295:253 296:251 297:235 298:27 321:64 322:251 323:253 324:251 325:126 349:64 350:251 351:253 352:251 353:126 377:64 378:251 379:253 380:251 381:126 405:64 406:253 407:255 408:221 433:182 434:251 435:253 436:200 460:64 461:236 462:251 463:253 464:62 487:8 488:158 489:251 490:251 491:169 492:8 515:32 516:251 517:251 518:251 519:158 543:32 544:253 545:253 546:253 547:159 571:32 572:251 573:251 574:251 575:39 599:32 600:251 601:251 602:251 627:32 628:251 629:251 630:251 631:100 655:32 656:251 657:251 658:251 +0 101:88 102:127 103:5 126:19 127:58 128:20 129:14 130:217 131:19 152:7 153:146 154:247 155:253 156:235 157:27 158:84 159:81 180:126 181:253 182:164 183:19 184:15 187:156 188:9 208:214 209:222 210:34 215:234 216:58 235:59 236:254 237:116 243:235 244:58 263:141 264:251 265:72 271:151 272:140 291:224 292:233 299:136 300:223 319:254 320:218 327:136 328:253 347:254 348:135 355:136 356:253 374:23 375:255 376:114 383:137 384:231 402:98 403:254 404:122 411:136 412:155 430:98 431:254 432:106 439:166 440:155 458:98 459:254 460:128 467:234 468:193 486:98 487:254 488:135 494:61 495:248 496:118 515:255 516:238 517:18 521:13 522:224 523:254 524:58 543:201 544:253 545:128 546:2 548:5 549:150 550:253 551:167 552:9 571:18 572:226 573:253 574:49 575:31 576:156 577:253 578:228 579:13 600:147 601:253 602:243 603:241 604:254 605:227 606:43 628:5 629:126 630:245 631:253 632:231 633:46 +0 127:37 128:141 129:156 130:156 131:194 132:194 133:47 153:11 154:132 155:239 156:253 157:253 158:253 159:253 160:254 161:181 180:25 181:172 182:253 183:235 184:167 185:78 186:93 187:174 188:254 189:247 190:54 207:26 208:210 209:253 210:237 211:90 216:201 217:253 218:78 235:192 236:253 237:237 238:58 244:156 245:253 246:78 262:141 263:254 264:235 265:53 269:19 270:5 272:156 273:254 274:78 289:46 290:254 291:253 292:92 296:17 297:226 298:217 299:49 300:148 301:253 302:78 317:165 318:254 319:239 320:24 324:20 325:253 326:253 327:58 328:18 329:115 330:24 344:37 345:248 346:254 347:91 352:2 353:117 354:250 355:163 356:91 372:77 373:253 374:254 375:39 382:196 383:253 384:173 400:159 401:254 402:218 403:15 410:77 411:254 412:255 413:61 428:234 429:253 430:113 438:21 439:226 440:254 441:135 455:25 456:240 457:253 458:68 467:195 468:254 469:135 483:79 484:253 485:253 495:195 496:254 497:135 511:79 512:253 513:253 514:76 523:195 524:254 525:99 540:212 541:254 542:209 543:9 550:10 551:209 552:196 553:15 568:54 569:253 570:254 571:137 572:36 576:2 577:20 578:168 579:253 580:60 596:28 597:235 598:254 599:253 600:199 601:124 602:79 603:79 604:167 605:253 606:253 607:185 608:30 625:15 626:117 627:217 628:253 629:253 630:253 631:254 632:253 633:240 634:109 635:12 655:27 656:126 657:208 658:253 659:193 660:147 661:40 +0 154:32 155:134 156:218 157:254 158:254 159:254 160:217 161:84 176:44 177:208 178:215 179:156 180:35 181:119 182:236 183:246 184:136 185:91 186:69 187:151 188:249 189:246 190:78 203:44 204:230 205:254 206:254 207:254 208:254 209:254 210:196 211:48 216:60 217:224 218:210 219:24 231:118 232:254 233:202 234:19 235:201 236:254 237:181 238:9 245:35 246:233 247:168 259:193 260:223 261:34 263:59 264:163 265:236 266:15 274:140 275:205 276:8 286:60 287:254 288:176 293:38 302:54 303:237 304:80 314:59 315:254 316:93 331:131 332:200 342:59 343:240 344:24 359:79 360:214 370:59 371:234 387:67 388:248 389:54 398:59 399:234 416:235 417:58 426:60 427:235 443:79 444:255 445:59 454:59 455:251 456:66 471:79 472:250 473:54 482:59 483:254 484:108 499:146 500:214 510:5 511:203 512:187 513:3 526:4 527:188 528:199 539:118 540:254 541:57 554:96 555:254 556:117 567:16 568:237 569:224 570:14 581:14 582:187 583:206 584:8 596:88 597:252 598:186 599:16 608:16 609:187 610:252 611:125 625:100 626:254 627:237 628:94 629:24 635:13 636:214 637:254 638:166 653:3 654:57 655:215 656:248 657:241 658:235 659:197 660:137 661:137 662:137 663:231 664:238 665:155 666:25 684:57 685:155 686:246 687:254 688:254 689:254 690:254 691:147 692:36 +1 124:102 125:252 126:252 127:41 152:102 153:250 154:250 155:202 180:102 181:250 182:250 183:232 184:91 208:102 209:250 210:250 211:212 212:29 236:102 237:252 238:252 239:254 240:150 264:102 265:250 266:250 267:252 268:149 292:102 293:250 294:250 295:252 296:149 320:102 321:250 322:250 323:252 324:231 325:80 349:152 350:252 351:254 352:252 353:100 377:151 378:250 379:252 380:250 381:100 405:151 406:250 407:252 408:250 409:100 433:151 434:250 435:252 436:250 437:100 461:123 462:243 463:254 464:252 465:100 490:202 491:252 492:250 493:100 518:80 519:252 520:250 521:190 522:30 547:252 548:250 549:250 550:49 575:255 576:252 577:252 578:252 579:214 580:31 603:171 604:250 605:250 606:250 607:252 608:190 609:40 631:20 632:160 633:250 634:250 635:252 636:250 637:100 660:20 661:170 662:250 663:212 664:49 665:20 +0 124:20 125:121 126:197 127:253 128:64 151:23 152:200 153:252 154:252 155:252 156:184 157:6 178:25 179:197 180:252 181:252 182:252 183:252 184:253 185:228 186:107 187:15 205:26 206:196 207:252 208:252 209:252 210:252 211:252 212:253 213:252 214:252 215:219 216:178 217:21 233:186 234:252 235:238 236:94 237:67 238:224 239:217 240:53 241:109 242:245 243:252 244:252 245:213 246:63 260:98 261:242 262:252 263:101 266:39 267:31 270:109 271:128 272:241 273:252 274:207 275:97 287:17 288:230 289:252 290:241 291:56 300:109 301:252 302:252 303:229 304:17 314:13 315:192 316:252 317:243 318:96 328:25 329:127 330:252 331:252 332:120 342:121 343:252 344:252 345:165 357:125 358:252 359:252 360:246 361:70 370:190 371:252 372:252 373:39 385:26 386:210 387:252 388:252 389:119 398:255 399:253 400:159 414:22 415:209 416:253 417:183 426:253 427:252 428:103 443:34 444:252 445:252 454:253 455:252 456:26 471:27 472:252 473:252 482:253 483:252 484:168 485:13 499:70 500:252 501:209 510:147 511:252 512:252 513:75 526:68 527:233 528:252 529:119 538:121 539:252 540:252 541:189 542:40 552:15 553:82 554:231 555:252 556:214 557:31 566:38 567:135 568:248 569:252 570:231 571:145 572:41 573:41 574:41 575:41 576:20 577:24 578:37 579:83 580:194 581:252 582:252 583:212 584:33 596:83 597:213 598:252 599:252 600:252 601:252 602:252 603:252 604:204 605:213 606:243 607:252 608:252 609:252 610:212 611:34 625:34 626:140 627:238 628:248 629:252 630:252 631:252 632:253 633:252 634:252 635:241 636:238 637:238 638:75 656:82 657:119 658:119 659:119 660:120 661:119 662:119 663:19 +1 127:20 128:254 129:255 130:37 155:19 156:253 157:253 158:134 183:19 184:253 185:253 186:246 187:125 211:76 212:253 213:253 214:253 215:158 239:207 240:253 241:253 242:253 243:158 267:207 268:253 269:253 270:253 271:158 294:48 295:223 296:253 297:253 298:243 299:106 322:141 323:253 324:253 325:253 326:113 349:65 350:237 351:253 352:253 353:253 354:36 377:76 378:253 379:253 380:253 381:253 382:36 405:76 406:253 407:253 408:253 409:253 410:36 433:76 434:253 435:253 436:253 437:118 438:4 460:4 461:148 462:253 463:253 464:253 465:103 488:10 489:253 490:253 491:253 492:253 493:103 516:10 517:253 518:253 519:253 520:173 521:7 544:10 545:253 546:253 547:253 548:168 572:143 573:253 574:253 575:239 576:49 600:198 601:253 602:253 603:234 615:140 628:198 629:253 630:253 631:234 656:198 657:253 658:253 659:234 +0 235:40 236:37 238:7 239:77 240:137 241:136 242:136 243:136 244:136 245:40 246:6 261:16 262:135 263:254 264:233 266:152 267:215 268:96 269:140 270:155 271:118 272:230 273:254 274:158 275:68 288:19 289:164 290:254 291:114 294:235 295:140 301:99 302:230 303:254 304:186 305:14 315:70 316:226 317:242 318:121 322:104 323:195 324:38 330:33 331:179 332:253 333:140 342:41 343:241 344:198 345:43 359:24 360:209 361:223 370:164 371:250 372:66 388:136 389:253 398:254 399:158 416:136 417:215 426:255 427:76 442:5 443:127 444:246 445:133 454:254 455:122 469:5 470:150 471:247 472:91 473:9 482:254 483:165 495:13 496:79 497:194 498:216 499:84 510:111 511:251 512:87 519:16 520:25 521:40 522:107 523:186 524:213 525:117 526:25 538:14 539:185 540:235 541:142 542:23 546:91 547:157 548:231 549:207 550:126 551:49 569:143 570:195 571:255 572:254 573:254 574:244 575:157 576:76 599:39 600:39 601:39 602:33 +1 128:166 129:255 130:187 131:6 156:165 157:253 158:253 159:13 183:15 184:191 185:253 186:253 187:13 211:49 212:253 213:253 214:253 215:13 239:141 240:253 241:253 242:169 243:4 266:4 267:189 268:253 269:249 270:53 294:69 295:253 296:253 297:246 322:69 323:253 324:253 325:246 350:118 351:253 352:253 353:124 378:206 379:253 380:231 381:21 405:66 406:241 407:253 408:199 433:105 434:253 435:253 436:89 460:3 461:228 462:253 463:252 464:86 488:111 489:253 490:253 491:205 516:166 517:253 518:253 519:75 543:43 544:249 545:253 546:193 547:9 570:4 571:160 572:253 573:253 574:184 598:37 599:253 600:253 601:253 602:88 626:140 627:253 628:253 629:186 630:18 654:14 655:253 656:253 657:27 +1 128:117 129:128 155:2 156:199 157:127 183:81 184:254 185:87 211:116 212:254 213:48 239:175 240:234 241:18 266:5 267:230 268:168 294:80 295:255 296:142 322:80 323:255 324:142 350:80 351:251 352:57 378:129 379:239 406:164 407:209 433:28 434:245 435:159 461:64 462:254 463:144 489:84 490:254 491:80 517:143 518:254 519:30 544:3 545:225 546:200 572:48 573:254 574:174 600:48 601:254 602:174 628:93 629:254 630:129 656:53 657:234 658:41 +1 129:159 130:142 156:11 157:220 158:141 184:78 185:254 186:141 212:111 213:254 214:109 240:196 241:221 242:15 267:26 268:221 269:159 295:63 296:254 297:159 323:178 324:254 325:93 350:7 351:191 352:254 353:97 378:42 379:255 380:254 381:41 406:42 407:254 408:195 409:10 434:141 435:255 436:78 461:11 462:202 463:254 464:59 489:86 490:254 491:254 492:59 517:142 518:254 519:248 520:52 545:142 546:254 547:195 573:142 574:254 575:164 601:142 602:254 603:77 629:142 630:254 631:131 657:77 658:172 659:5 +0 124:66 125:254 126:254 127:58 128:60 129:59 130:59 131:50 151:73 152:233 153:253 154:253 155:148 156:254 157:253 158:253 159:232 160:73 179:156 180:253 181:253 182:253 183:117 184:255 185:253 186:253 187:253 188:223 189:176 190:162 205:37 206:116 207:246 208:253 209:180 210:18 211:4 212:18 213:109 214:241 215:253 216:253 217:253 218:236 219:28 233:235 234:253 235:253 236:245 237:107 242:109 243:170 244:253 245:253 246:253 247:174 261:235 262:253 263:253 264:233 271:15 272:156 273:253 274:253 275:223 276:72 287:10 288:156 289:250 290:253 291:253 292:67 300:99 301:253 302:253 303:253 304:127 305:5 315:118 316:253 317:253 318:253 319:204 320:26 328:68 329:223 330:253 331:253 332:253 333:57 342:32 343:191 344:253 345:253 346:253 347:97 357:156 358:253 359:253 360:253 361:57 370:59 371:253 372:253 373:253 374:253 375:97 385:36 386:224 387:253 388:253 389:57 398:60 399:254 400:255 401:254 402:156 413:37 414:226 415:254 416:254 417:58 426:59 427:253 428:253 429:253 430:154 441:156 442:253 443:253 444:253 445:57 454:59 455:253 456:253 457:253 458:154 469:156 470:253 471:253 472:253 473:57 482:59 483:253 484:253 485:253 486:246 487:90 496:16 497:171 498:253 499:253 500:231 501:49 510:59 511:253 512:253 513:253 514:253 515:156 516:91 524:99 525:253 526:253 527:222 528:71 538:59 539:253 540:253 541:253 542:253 543:253 544:245 545:109 551:145 552:194 553:253 554:253 555:174 566:9 567:38 568:174 569:251 570:253 571:253 572:253 573:241 574:215 575:215 576:217 577:215 578:215 579:250 580:253 581:253 582:221 583:26 597:235 598:253 599:253 600:253 601:253 602:253 603:253 604:254 605:253 606:253 607:253 608:253 609:204 610:26 625:108 626:116 627:200 628:253 629:253 630:253 631:253 632:254 633:253 634:253 635:253 636:199 637:44 655:36 656:57 657:118 658:253 659:253 660:58 661:57 662:57 663:57 664:35 +1 129:101 130:222 131:84 157:225 158:252 159:84 184:89 185:246 186:208 187:19 212:128 213:252 214:195 239:79 240:253 241:252 242:195 267:141 268:255 269:253 270:133 294:26 295:240 296:253 297:252 298:55 322:60 323:252 324:253 325:154 326:12 349:7 350:178 351:252 352:253 353:27 377:57 378:252 379:252 380:253 381:27 405:57 406:253 407:253 408:204 409:15 433:104 434:252 435:252 436:94 460:19 461:209 462:252 463:252 488:101 489:252 490:252 491:157 516:225 517:252 518:252 519:112 544:226 545:253 546:240 547:63 572:225 573:252 574:223 600:225 601:252 602:223 628:225 629:252 630:242 631:75 656:146 657:252 658:236 659:50 +0 124:41 125:254 126:254 127:157 128:34 129:34 130:218 131:255 132:206 133:34 134:18 151:53 152:238 153:252 154:252 155:252 156:252 157:252 158:252 159:252 160:252 161:252 162:162 163:26 178:66 179:220 180:252 181:252 182:252 183:209 184:153 185:223 186:252 187:252 188:252 189:252 190:252 191:98 206:166 207:252 208:252 209:252 210:252 211:141 213:85 214:230 215:252 216:252 217:252 218:252 219:98 234:166 235:252 236:252 237:252 238:252 239:141 242:73 243:102 244:252 245:252 246:252 247:98 262:166 263:252 264:252 265:252 266:191 267:30 271:5 272:97 273:252 274:252 275:220 276:51 289:123 290:245 291:252 292:252 293:202 294:14 300:56 301:252 302:252 303:252 304:65 316:18 317:154 318:252 319:252 320:241 328:56 329:252 330:252 331:252 332:65 343:21 344:146 345:252 346:252 347:252 348:241 356:56 357:252 358:252 359:252 360:65 371:67 372:252 373:252 374:252 375:252 376:241 384:56 385:252 386:252 387:252 388:65 399:67 400:252 401:252 402:252 403:252 404:116 412:56 413:252 414:252 415:252 416:65 427:67 428:252 429:252 430:252 431:252 432:20 440:56 441:252 442:252 443:252 444:65 455:67 456:252 457:252 458:252 459:87 460:4 468:56 469:252 470:252 471:124 472:11 483:67 484:252 485:252 486:252 487:54 494:19 495:236 496:245 497:252 498:252 499:98 511:67 512:252 513:252 514:252 515:97 516:5 521:39 522:219 523:252 524:252 525:252 526:252 527:98 539:67 540:252 541:252 542:252 543:252 544:102 545:89 546:89 547:89 548:89 549:203 550:252 551:252 552:252 553:252 554:209 555:64 567:67 568:252 569:252 570:252 571:252 572:252 573:252 574:252 575:252 576:252 577:252 578:252 579:252 580:226 581:130 582:68 595:67 596:252 597:252 598:252 599:252 600:252 601:252 602:252 603:252 604:252 605:252 606:252 607:239 608:77 623:17 624:65 625:163 626:252 627:252 628:252 629:252 630:252 631:252 632:252 633:252 634:96 635:59 653:17 654:176 655:252 656:252 657:252 658:252 659:155 660:32 661:32 662:6 +0 96:56 97:247 98:121 124:24 125:242 126:245 127:122 153:231 154:253 155:253 156:104 157:12 181:90 182:253 183:253 184:254 185:221 186:120 187:120 188:85 206:67 207:75 208:36 209:11 210:56 211:222 212:254 213:253 214:253 215:253 216:245 217:207 218:36 233:86 234:245 235:249 236:105 239:44 240:224 241:230 242:253 243:253 244:253 245:253 246:214 247:10 260:8 261:191 262:253 263:143 269:29 270:119 271:119 272:158 273:253 274:253 275:94 288:15 289:253 290:226 291:48 300:4 301:183 302:253 303:248 304:56 316:42 317:253 318:178 329:179 330:253 331:184 332:14 344:164 345:253 346:178 357:179 358:253 359:163 371:61 372:254 373:254 374:179 384:76 385:254 386:254 387:164 399:60 400:253 401:253 402:178 411:29 412:206 413:253 414:253 415:40 427:60 428:253 429:253 430:178 439:120 440:253 441:253 442:245 443:13 455:60 456:253 457:253 458:178 467:120 468:253 469:239 470:63 483:60 484:253 485:253 486:178 494:14 495:238 496:253 497:179 511:18 512:190 513:253 514:231 515:70 521:43 522:184 523:253 524:253 525:74 540:86 541:253 542:253 543:239 544:134 545:8 548:56 549:163 550:253 551:253 552:213 553:35 568:16 569:253 570:253 571:253 572:253 573:240 574:239 575:239 576:247 577:253 578:253 579:210 580:27 596:4 597:59 598:204 599:253 600:253 601:253 602:253 603:253 604:254 605:253 606:250 607:110 626:31 627:122 628:253 629:253 630:253 631:253 632:255 633:217 634:98 +0 125:19 126:164 127:253 128:255 129:253 130:118 131:59 132:36 153:78 154:251 155:251 156:253 157:251 158:251 159:251 160:199 161:45 180:14 181:198 182:251 183:251 184:253 185:251 186:251 187:251 188:251 189:204 190:26 208:5 209:117 210:251 211:251 212:243 213:212 214:239 215:251 216:251 217:251 218:218 236:95 237:251 238:251 239:251 240:120 242:175 243:251 244:251 245:251 246:231 263:97 264:237 265:251 266:251 267:251 270:67 271:240 272:251 273:251 274:243 275:108 290:8 291:163 292:251 293:251 294:240 295:81 299:68 300:251 301:251 302:251 303:179 304:9 317:13 318:145 319:251 320:251 321:226 322:80 327:39 328:251 329:251 330:251 331:251 332:115 345:144 346:251 347:251 348:251 349:173 355:18 356:167 357:251 358:251 359:251 360:115 373:233 374:251 375:251 376:251 377:173 384:98 385:251 386:251 387:251 388:115 400:176 401:253 402:253 403:216 404:179 412:99 413:253 414:253 415:253 416:116 427:55 428:210 429:251 430:251 431:96 440:98 441:251 442:251 443:214 444:62 455:117 456:251 457:251 458:251 459:96 467:28 468:204 469:251 470:237 471:53 482:55 483:241 484:251 485:251 486:160 487:7 494:28 495:222 496:251 497:251 498:231 510:59 511:251 512:251 513:251 514:153 520:23 521:98 522:204 523:251 524:251 525:251 526:156 538:59 539:251 540:251 541:251 542:153 546:85 547:155 548:179 549:251 550:251 551:251 552:251 553:154 554:15 566:59 567:251 568:251 569:251 570:236 571:214 572:214 573:214 574:234 575:251 576:253 577:251 578:251 579:248 580:156 581:15 594:41 595:209 596:251 597:251 598:251 599:251 600:251 601:251 602:251 603:251 604:253 605:251 606:196 607:146 623:54 624:115 625:241 626:251 627:251 628:251 629:251 630:251 631:251 632:253 633:187 634:35 653:83 654:251 655:251 656:251 657:251 658:251 659:101 660:57 661:31 +1 129:232 130:255 131:107 156:58 157:244 158:253 159:106 184:95 185:253 186:253 187:106 212:95 213:253 214:253 215:106 240:95 241:253 242:249 243:69 268:144 269:253 270:192 295:97 296:233 297:253 298:66 323:195 324:253 325:253 326:5 350:38 351:232 352:253 353:182 354:2 377:10 378:160 379:253 380:231 381:53 405:42 406:253 407:253 408:158 433:141 434:253 435:253 436:115 460:75 461:245 462:253 463:183 464:4 487:1 488:147 489:253 490:251 491:58 515:20 516:253 517:253 518:180 543:202 544:253 545:226 546:27 571:243 572:253 573:212 598:85 599:251 600:253 601:173 626:209 627:253 628:244 629:57 654:169 655:253 656:174 +1 127:63 128:128 129:2 155:63 156:254 157:123 183:63 184:254 185:179 211:63 212:254 213:179 239:63 240:254 241:179 267:142 268:254 269:179 295:187 296:254 297:158 323:187 324:254 325:55 350:68 351:235 352:254 353:55 378:181 379:254 380:254 381:55 406:181 407:254 408:202 409:14 434:181 435:254 436:186 462:181 463:254 464:146 490:181 491:254 492:62 518:181 519:254 520:62 546:181 547:254 548:62 574:181 575:255 576:62 602:181 603:254 604:241 605:52 630:181 631:254 632:222 633:30 658:181 659:224 660:34 +1 130:131 131:255 132:184 133:15 157:99 158:247 159:253 160:182 161:15 185:124 186:253 187:253 188:253 189:38 212:9 213:171 214:253 215:253 216:140 217:1 240:47 241:253 242:253 243:251 244:117 267:43 268:219 269:253 270:253 271:153 295:78 296:253 297:253 298:253 299:84 323:97 324:253 325:253 326:244 327:74 350:69 351:243 352:253 353:253 354:183 377:10 378:168 379:253 380:253 381:215 382:34 405:31 406:253 407:253 408:253 409:129 433:107 434:253 435:253 436:242 437:67 460:24 461:204 462:253 463:253 464:187 488:95 489:253 490:253 491:201 492:25 516:239 517:253 518:253 519:176 543:119 544:251 545:253 546:253 547:138 570:30 571:212 572:253 573:252 574:165 575:8 598:193 599:253 600:253 601:222 626:193 627:253 628:253 629:189 654:193 655:253 656:201 657:27 +0 125:57 126:255 127:253 128:198 129:85 153:168 154:253 155:251 156:253 157:251 158:169 159:56 180:86 181:253 182:254 183:253 184:254 185:253 186:254 187:253 188:57 208:197 209:251 210:253 211:251 212:253 213:251 214:253 215:251 216:225 217:56 235:169 236:255 237:253 238:226 239:56 241:114 242:254 243:253 244:254 245:84 262:57 263:224 264:253 265:251 266:56 270:139 271:251 272:253 273:83 290:141 291:253 292:255 293:84 298:57 299:225 300:254 301:196 318:253 319:251 320:253 321:83 327:168 328:253 329:83 345:169 346:254 347:253 348:169 355:169 356:254 357:253 358:169 373:168 374:253 375:251 376:56 383:168 384:253 385:251 386:56 401:169 402:254 403:84 412:254 413:253 429:168 430:253 431:83 440:253 441:251 456:29 457:197 458:254 459:84 467:169 468:254 469:196 484:85 485:251 486:253 487:83 494:57 495:224 496:253 497:83 512:57 513:225 514:254 515:139 521:57 522:141 523:253 524:254 525:84 541:168 542:253 543:251 544:169 545:56 547:114 548:169 549:224 550:253 551:251 552:253 553:83 569:169 570:254 571:253 572:254 573:253 574:254 575:253 576:254 577:253 578:254 579:253 580:226 581:56 597:56 598:253 599:251 600:253 601:251 602:253 603:251 604:253 605:251 606:253 607:251 608:56 626:169 627:225 628:254 629:253 630:254 631:253 632:254 633:253 634:226 635:56 655:56 656:253 657:251 658:253 659:251 660:84 661:83 662:56 +0 127:12 128:105 129:224 130:255 131:247 132:22 155:131 156:254 157:254 158:243 159:252 160:76 182:131 183:225 184:254 185:224 186:48 187:136 208:13 209:109 210:252 211:254 212:254 213:254 214:197 215:76 235:9 236:181 237:254 238:254 239:240 240:229 241:237 242:254 243:252 244:152 245:21 262:9 263:143 264:254 265:254 266:226 267:36 269:22 270:138 271:254 272:254 273:188 289:13 290:181 291:254 292:254 293:250 294:64 298:2 299:53 300:236 301:252 302:131 317:102 318:254 319:254 320:254 321:111 328:56 329:243 330:251 331:42 344:30 345:186 346:254 347:254 348:206 349:29 357:199 358:254 359:91 372:92 373:254 374:254 375:237 376:13 385:134 386:254 387:91 400:133 401:254 402:254 403:126 413:134 414:250 415:17 428:187 429:254 430:237 431:23 441:200 442:183 456:187 457:254 458:213 467:2 468:134 469:252 470:101 484:183 485:254 486:133 495:14 496:254 497:234 498:34 512:92 513:254 514:161 522:84 523:204 524:254 525:56 540:92 541:254 542:229 549:85 550:252 551:252 552:188 553:11 568:56 569:252 570:229 575:3 576:53 577:235 578:253 579:166 597:224 598:245 599:130 600:68 601:68 602:134 603:214 604:254 605:254 606:159 625:141 626:254 627:254 628:254 629:254 630:254 631:254 632:233 633:95 634:3 653:14 654:152 655:254 656:254 657:254 658:186 659:157 660:53 +1 130:226 131:247 132:55 157:99 158:248 159:254 160:230 161:30 185:125 186:254 187:254 188:254 189:38 213:125 214:254 215:254 216:212 217:24 240:18 241:223 242:254 243:252 244:118 268:24 269:254 270:254 271:239 295:27 296:195 297:254 298:254 299:93 323:78 324:254 325:254 326:246 327:74 351:158 352:254 353:254 354:185 378:41 379:239 380:254 381:254 382:43 405:22 406:218 407:254 408:254 409:167 410:9 433:32 434:254 435:254 436:254 437:130 460:24 461:187 462:254 463:254 464:234 465:16 488:189 489:254 490:254 491:254 492:128 515:64 516:247 517:254 518:255 519:219 520:42 543:139 544:254 545:254 546:222 547:40 570:30 571:213 572:254 573:235 574:45 598:194 599:254 600:254 601:223 626:194 627:254 628:254 629:190 654:194 655:254 656:202 657:27 +1 130:166 131:253 132:124 133:53 158:140 159:251 160:251 161:180 185:125 186:246 187:251 188:251 189:51 212:32 213:190 214:251 215:251 216:251 217:103 240:21 241:174 242:251 243:251 244:251 268:73 269:176 270:253 271:253 272:201 296:149 297:251 298:251 299:251 300:71 323:27 324:228 325:251 326:251 327:157 328:10 351:180 352:253 353:251 354:251 355:142 377:27 378:180 379:231 380:253 381:251 382:96 383:41 405:89 406:253 407:253 408:255 409:211 410:25 433:217 434:251 435:251 436:253 437:107 460:21 461:221 462:251 463:251 464:242 465:92 487:32 488:190 489:251 490:251 491:251 492:103 515:202 516:251 517:251 518:251 519:122 542:53 543:255 544:253 545:253 546:221 547:51 570:180 571:253 572:251 573:251 574:142 598:180 599:253 600:251 601:251 602:142 626:180 627:253 628:251 629:157 630:82 654:180 655:253 656:147 657:10 +1 129:17 130:206 131:229 132:44 157:2 158:125 159:254 160:123 185:95 186:254 187:254 188:123 212:78 213:240 214:254 215:254 216:123 240:100 241:254 242:254 243:254 244:123 267:2 268:129 269:254 270:254 271:220 272:20 295:9 296:254 297:254 298:254 299:123 322:22 323:179 324:254 325:254 326:254 327:49 350:83 351:254 352:254 353:254 354:183 355:19 378:136 379:254 380:254 381:254 382:139 404:3 405:111 406:252 407:254 408:254 409:232 410:45 432:67 433:254 434:254 435:254 436:216 437:40 459:14 460:192 461:254 462:254 463:254 464:140 486:23 487:192 488:254 489:254 490:254 491:246 514:77 515:254 516:254 517:255 518:241 519:100 541:65 542:235 543:254 544:254 545:254 546:172 568:30 569:238 570:254 571:254 572:254 573:219 574:26 596:34 597:254 598:254 599:254 600:216 601:41 624:34 625:254 626:254 627:254 628:188 652:12 653:170 654:254 655:254 656:82 +1 130:218 131:253 132:124 157:84 158:236 159:251 160:251 184:63 185:236 186:251 187:251 188:122 212:73 213:251 214:251 215:251 216:173 240:202 241:251 242:251 243:251 244:71 267:53 268:255 269:253 270:253 271:253 272:72 295:180 296:253 297:251 298:251 299:188 300:30 323:180 324:253 325:251 326:251 327:142 350:47 351:211 352:253 353:251 354:235 355:82 377:27 378:211 379:251 380:253 381:251 382:215 405:89 406:253 407:253 408:255 409:253 410:164 433:217 434:251 435:251 436:253 437:168 438:15 460:21 461:221 462:251 463:251 464:253 465:107 487:32 488:190 489:251 490:251 491:251 492:221 493:61 515:73 516:251 517:251 518:251 519:251 520:180 543:255 544:253 545:253 546:253 547:201 570:105 571:253 572:251 573:251 574:251 575:71 598:180 599:253 600:251 601:246 602:137 603:10 626:180 627:253 628:251 629:215 654:180 655:253 656:251 657:86 +1 124:102 125:180 126:1 152:140 153:254 154:130 180:140 181:254 182:204 208:140 209:254 210:204 236:72 237:254 238:204 264:25 265:231 266:250 267:135 292:11 293:211 294:254 295:222 321:101 322:254 323:250 324:15 349:96 350:254 351:254 352:95 377:2 378:251 379:254 380:95 405:2 406:251 407:254 408:95 433:96 434:254 435:254 436:95 461:53 462:253 463:254 464:139 490:250 491:254 492:235 493:27 518:201 519:254 520:254 521:128 546:80 547:254 548:254 549:139 574:65 575:254 576:254 577:139 602:150 603:254 604:254 605:139 630:229 631:254 632:254 633:43 658:52 659:196 660:168 661:9 +0 128:87 129:208 130:249 155:27 156:212 157:254 158:195 182:118 183:225 184:254 185:254 186:232 187:147 188:46 209:115 210:248 211:254 212:254 213:254 214:254 215:254 216:230 217:148 218:12 236:18 237:250 238:254 239:245 240:226 241:254 242:254 243:254 244:254 245:254 246:148 263:92 264:205 265:254 266:250 267:101 268:20 269:194 270:254 271:254 272:254 273:254 274:229 275:53 291:152 292:254 293:254 294:94 297:14 298:124 299:187 300:254 301:254 302:254 303:213 318:95 319:252 320:254 321:206 322:15 327:3 328:6 329:51 330:231 331:254 332:94 345:50 346:246 347:254 348:254 349:20 358:200 359:254 360:96 372:21 373:184 374:254 375:254 376:147 377:2 386:200 387:254 388:96 400:177 401:254 402:254 403:218 404:33 413:16 414:211 415:254 416:96 427:11 428:219 429:254 430:251 431:92 441:84 442:254 443:232 444:44 455:101 456:254 457:254 458:141 469:162 470:254 471:231 472:42 483:235 484:254 485:227 486:42 496:51 497:238 498:254 499:213 511:235 512:254 513:199 524:160 525:254 526:229 527:52 539:235 540:254 541:199 549:10 550:84 551:150 552:253 553:254 554:147 567:235 568:254 569:213 570:20 575:17 576:63 577:158 578:254 579:254 580:254 581:155 582:12 595:122 596:248 597:254 598:204 599:98 600:42 601:177 602:180 603:200 604:254 605:254 606:253 607:213 608:82 609:10 624:203 625:254 626:254 627:254 628:254 629:254 630:254 631:254 632:251 633:219 634:94 652:35 653:221 654:254 655:254 656:254 657:254 658:254 659:217 660:95 +1 126:134 127:230 154:133 155:231 156:10 182:133 183:253 184:96 210:133 211:253 212:96 238:133 239:253 240:183 266:133 267:253 268:217 294:133 295:253 296:217 322:133 323:253 324:217 350:133 351:253 352:217 378:133 379:253 380:217 406:134 407:254 408:218 434:133 435:253 436:159 462:133 463:253 464:199 490:156 491:253 492:96 518:254 519:247 520:73 546:254 547:248 548:74 573:99 574:254 575:245 576:64 600:89 601:230 602:254 603:125 627:140 628:251 629:253 630:243 631:10 655:114 656:242 657:195 658:69 +1 125:29 126:85 127:255 128:139 153:197 154:251 155:253 156:251 181:254 182:253 183:254 184:253 209:253 210:251 211:253 212:251 237:254 238:253 239:254 240:253 265:253 266:251 267:253 268:138 293:254 294:253 295:254 296:196 321:253 322:251 323:253 324:196 349:254 350:253 351:254 352:84 377:253 378:251 379:253 380:196 405:254 406:253 407:254 408:253 433:253 434:251 435:253 436:251 461:254 462:253 463:254 464:253 489:253 490:251 491:253 492:251 517:254 518:253 519:254 520:253 545:253 546:251 547:253 548:251 573:254 574:253 575:254 576:253 601:253 602:251 603:253 604:251 629:57 630:225 631:254 632:253 658:56 659:253 660:251 +1 125:149 126:255 127:254 128:58 153:215 154:253 155:183 156:2 180:41 181:232 182:253 183:181 208:92 209:253 210:253 211:181 236:92 237:253 238:253 239:181 264:92 265:253 266:253 267:181 292:92 293:253 294:253 295:181 320:92 321:253 322:253 323:181 348:92 349:253 350:253 351:181 376:92 377:253 378:253 379:181 404:92 405:253 406:253 407:181 432:92 433:253 434:253 435:181 460:92 461:253 462:253 463:181 488:31 489:228 490:253 491:181 517:198 518:253 519:228 520:54 545:33 546:226 547:253 548:195 549:7 574:199 575:253 576:253 577:75 602:34 603:218 604:253 605:228 606:117 607:14 608:12 631:33 632:219 633:253 634:253 635:253 636:211 660:32 661:123 662:149 663:230 664:41 +1 130:79 131:203 132:141 157:51 158:240 159:240 160:140 185:88 186:252 187:252 188:140 213:197 214:252 215:252 216:140 241:197 242:252 243:252 244:140 268:147 269:253 270:253 271:253 295:38 296:234 297:252 298:242 299:89 323:113 324:252 325:252 326:223 350:16 351:207 352:252 353:252 354:129 377:16 378:203 379:253 380:252 381:220 382:37 405:29 406:253 407:255 408:253 409:56 432:19 433:181 434:252 435:253 436:176 437:6 460:166 461:252 462:252 463:228 464:52 487:10 488:203 489:252 490:252 491:126 514:63 515:178 516:252 517:252 518:173 542:114 543:253 544:253 545:225 570:238 571:252 572:252 573:99 596:7 597:135 598:253 599:252 600:176 601:19 624:29 625:252 626:253 627:252 628:55 652:13 653:189 654:253 655:204 656:25 +1 126:94 127:254 128:75 154:166 155:253 156:231 182:208 183:253 184:147 210:208 211:253 212:116 238:208 239:253 240:168 266:146 267:254 268:222 294:166 295:253 296:116 322:208 323:253 324:116 350:166 351:253 352:158 378:145 379:253 380:231 406:209 407:254 408:169 434:187 435:253 436:168 462:93 463:253 464:116 490:93 491:253 492:116 518:93 519:253 520:116 546:94 547:254 548:179 549:11 574:93 575:253 576:246 577:101 602:145 603:253 604:255 605:92 630:93 631:253 632:246 633:59 658:93 659:253 660:74 +0 127:46 128:105 129:254 130:254 131:224 132:59 133:59 134:9 155:196 156:254 157:253 158:253 159:253 160:253 161:253 162:128 182:96 183:235 184:254 185:253 186:253 187:253 188:253 189:253 190:247 191:122 208:4 209:101 210:244 211:253 212:254 213:234 214:241 215:253 216:253 217:253 218:253 219:186 220:18 236:96 237:253 238:253 239:253 240:232 241:83 242:109 243:170 244:253 245:253 246:253 247:253 248:116 264:215 265:253 266:253 267:253 268:196 271:40 272:253 273:253 274:253 275:253 276:116 290:8 291:141 292:247 293:253 294:253 295:237 296:29 299:6 300:38 301:171 302:253 303:253 304:116 317:13 318:146 319:253 320:253 321:253 322:253 323:57 329:156 330:253 331:253 332:116 345:40 346:253 347:253 348:253 349:253 350:178 351:27 357:156 358:253 359:253 360:116 372:136 373:204 374:253 375:253 376:253 377:192 378:27 385:156 386:253 387:253 388:116 399:28 400:195 401:254 402:254 403:254 404:250 405:135 412:99 413:255 414:254 415:254 416:117 427:118 428:253 429:253 430:253 431:253 432:142 439:19 440:170 441:253 442:253 443:216 444:62 454:42 455:212 456:253 457:253 458:253 459:253 460:38 466:124 467:188 468:253 469:253 470:253 471:174 482:59 483:253 484:253 485:253 486:237 487:93 488:3 491:31 492:40 493:130 494:247 495:253 496:253 497:253 498:204 499:13 510:59 511:253 512:253 513:253 514:154 518:54 519:218 520:254 521:253 522:253 523:253 524:253 525:253 526:38 538:59 539:253 540:253 541:253 542:215 543:156 544:156 545:156 546:209 547:253 548:255 549:253 550:253 551:253 552:192 553:97 554:15 566:55 567:242 568:253 569:253 570:253 571:253 572:253 573:253 574:253 575:253 576:254 577:253 578:253 579:204 580:23 595:118 596:253 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:254 605:216 606:174 607:13 623:54 624:116 625:243 626:253 627:253 628:253 629:253 630:253 631:146 632:117 633:62 653:53 654:132 655:253 656:253 657:192 658:57 659:13 +1 125:42 126:232 127:254 128:58 153:86 154:253 155:253 156:58 181:86 182:253 183:253 184:58 209:206 210:253 211:253 212:58 237:215 238:253 239:253 240:58 265:215 266:253 267:253 268:58 293:215 294:253 295:253 296:58 321:215 322:253 323:253 324:58 349:215 350:253 351:253 352:58 377:215 378:253 379:253 380:58 405:215 406:253 407:253 408:58 433:188 434:253 435:253 436:85 461:86 462:253 463:253 464:200 465:12 489:29 490:223 491:253 492:253 493:151 518:209 519:253 520:253 521:194 546:128 547:253 548:253 549:200 550:8 574:32 575:213 576:253 577:253 578:152 579:6 603:32 604:221 605:253 606:253 607:153 608:5 632:90 633:215 634:253 635:253 636:151 661:59 662:253 663:253 664:84 +1 156:60 157:229 158:38 184:187 185:254 186:78 211:121 212:252 213:254 214:78 239:197 240:254 241:206 242:6 267:197 268:254 269:202 294:27 295:218 296:233 297:62 322:117 323:254 324:195 350:203 351:254 352:195 377:64 378:244 379:254 380:195 405:79 406:254 407:255 408:161 433:79 434:254 435:254 436:65 461:79 462:254 463:241 464:52 489:79 490:254 491:189 517:79 518:254 519:189 545:79 546:254 547:189 573:79 574:254 575:189 601:79 602:254 603:194 604:5 629:35 630:219 631:254 632:72 658:34 659:223 660:195 687:129 688:195 +1 101:11 102:150 103:72 129:37 130:251 131:71 157:63 158:251 159:71 185:217 186:251 187:71 213:217 214:251 215:71 240:145 241:253 242:253 243:72 267:42 268:206 269:251 270:251 271:71 295:99 296:251 297:251 298:251 299:71 323:253 324:251 325:251 326:251 327:71 350:130 351:253 352:251 353:251 354:251 355:71 377:110 378:253 379:255 380:253 381:253 382:253 383:72 405:109 406:251 407:253 408:251 409:251 410:188 411:30 433:109 434:251 435:253 436:251 437:246 438:123 460:16 461:170 462:251 463:253 464:251 465:215 488:37 489:251 490:251 491:253 492:251 493:86 516:218 517:253 518:253 519:255 520:253 521:35 543:84 544:236 545:251 546:251 547:253 548:168 549:15 571:144 572:251 573:251 574:251 575:190 576:15 599:144 600:251 601:251 602:251 603:180 626:53 627:221 628:251 629:251 630:251 631:180 +0 127:45 128:254 129:254 130:254 131:148 132:24 133:9 154:43 155:254 156:252 157:252 158:252 159:252 160:252 161:121 162:13 181:58 182:237 183:254 184:252 185:252 186:252 187:252 188:252 189:252 190:68 208:69 209:224 210:252 211:254 212:252 213:252 214:252 215:252 216:252 217:252 218:135 219:17 235:75 236:216 237:252 238:252 239:254 240:231 241:168 242:252 243:252 244:252 245:252 246:252 247:45 262:77 263:212 264:252 265:252 266:252 267:242 268:93 269:32 270:114 271:177 272:252 273:252 274:252 275:158 276:12 289:75 290:212 291:252 292:252 293:252 294:252 295:231 299:116 300:252 301:252 302:252 303:252 304:21 316:69 317:216 318:252 319:252 320:252 321:252 322:252 323:62 327:116 328:252 329:252 330:252 331:252 332:21 344:93 345:252 346:252 347:252 348:252 349:252 350:62 355:21 356:158 357:252 358:252 359:252 360:21 371:64 372:239 373:252 374:252 375:252 376:252 377:252 378:21 384:139 385:252 386:252 387:252 388:21 398:5 399:87 400:254 401:254 402:254 403:254 404:237 405:41 411:11 412:150 413:254 414:254 415:254 416:22 425:5 426:85 427:252 428:252 429:252 430:252 431:222 432:55 439:116 440:252 441:252 442:252 443:214 444:18 453:24 454:252 455:252 456:252 457:252 458:252 459:91 466:26 467:153 468:252 469:252 470:252 471:45 481:24 482:252 483:252 484:252 485:252 486:252 487:91 492:18 493:93 494:151 495:252 496:252 497:252 498:184 499:28 509:24 510:252 511:252 512:252 513:252 514:252 515:164 516:116 517:116 518:116 519:117 520:141 521:252 522:252 523:252 524:252 525:252 526:68 537:24 538:252 539:252 540:252 541:252 542:252 543:252 544:252 545:252 546:252 547:254 548:252 549:252 550:252 551:252 552:252 553:163 554:31 565:9 566:121 567:252 568:252 569:252 570:252 571:252 572:252 573:252 574:252 575:254 576:252 577:252 578:252 579:178 580:91 581:33 594:13 595:119 596:252 597:252 598:252 599:252 600:252 601:252 602:252 603:254 604:252 605:252 606:184 607:37 623:13 624:121 625:252 626:252 627:252 628:252 629:252 630:252 631:254 632:214 633:45 634:28 652:8 653:21 654:21 655:169 656:252 657:252 658:41 659:22 660:18 +0 125:218 126:253 127:253 128:255 129:149 130:62 151:42 152:144 153:236 154:251 155:251 156:253 157:251 158:236 159:144 160:144 179:99 180:251 181:251 182:251 183:225 184:253 185:251 186:251 187:251 188:251 189:166 190:16 206:79 207:253 208:251 209:251 210:204 211:41 212:143 213:205 214:251 215:251 216:251 217:253 218:169 219:15 233:79 234:231 235:253 236:251 237:225 238:41 241:41 242:226 243:251 244:251 245:253 246:251 247:164 260:37 261:253 262:253 263:255 264:253 265:35 271:79 272:232 273:255 274:253 275:227 276:42 288:140 289:251 290:251 291:253 292:168 293:15 300:77 301:253 302:251 303:251 304:142 315:21 316:221 317:251 318:251 319:164 320:15 329:227 330:251 331:251 332:236 333:61 342:32 343:190 344:251 345:251 346:251 357:73 358:251 359:251 360:251 361:71 370:73 371:251 372:251 373:251 374:251 385:73 386:251 387:251 388:251 389:71 398:73 399:253 400:253 401:253 402:201 413:73 414:253 415:253 416:253 417:72 426:176 427:251 428:251 429:251 430:71 441:73 442:251 443:251 444:251 445:71 454:253 455:251 456:251 457:157 458:10 469:73 470:251 471:251 472:251 473:71 482:253 483:251 484:251 485:142 497:150 498:251 499:251 500:204 501:41 510:124 511:251 512:251 513:220 514:180 524:130 525:253 526:251 527:225 528:41 538:73 539:253 540:253 541:253 542:253 543:73 544:73 545:10 549:42 550:73 551:150 552:253 553:255 554:253 555:216 566:31 567:189 568:251 569:251 570:251 571:253 572:251 573:159 574:144 575:144 576:145 577:206 578:251 579:251 580:251 581:253 582:168 583:92 595:20 596:195 597:251 598:251 599:253 600:251 601:251 602:251 603:251 604:253 605:251 606:251 607:251 608:225 609:164 610:15 624:21 625:142 626:220 627:253 628:251 629:251 630:251 631:251 632:253 633:251 634:251 635:204 636:41 654:51 655:72 656:174 657:251 658:251 659:251 660:253 661:147 662:71 663:41 +0 127:60 128:96 129:96 130:48 153:16 154:171 155:228 156:253 157:251 158:220 159:51 160:32 181:127 182:251 183:251 184:253 185:251 186:251 187:251 188:251 189:80 207:24 208:182 209:236 210:251 211:211 212:189 213:236 214:251 215:251 216:251 217:242 218:193 234:100 235:194 236:251 237:251 238:211 239:35 241:71 242:173 243:251 244:251 245:253 246:240 247:158 248:19 261:64 262:253 263:255 264:253 265:205 266:19 271:40 272:218 273:255 274:253 275:253 276:91 288:16 289:186 290:251 291:253 292:247 293:110 300:39 301:233 302:251 303:251 304:188 315:16 316:189 317:251 318:251 319:205 320:110 329:48 330:220 331:251 332:220 333:48 343:72 344:251 345:251 346:251 347:158 358:51 359:251 360:251 361:232 371:190 372:251 373:251 374:251 375:59 386:32 387:251 388:251 389:251 398:96 399:253 400:253 401:253 402:95 414:32 415:253 416:253 417:193 426:214 427:251 428:251 429:204 430:23 442:52 443:251 444:251 445:94 454:253 455:251 456:251 457:109 469:48 470:221 471:251 472:219 473:47 482:253 483:251 484:251 485:70 497:234 498:251 499:251 500:188 510:253 511:251 512:251 513:188 523:40 524:158 525:253 526:251 527:172 528:70 539:191 540:253 541:253 542:253 543:96 544:24 549:12 550:174 551:253 552:253 553:255 554:221 567:71 568:251 569:251 570:251 571:253 572:205 573:190 574:190 575:190 576:191 577:197 578:251 579:251 580:231 581:221 582:93 595:16 596:126 597:236 598:251 599:253 600:251 601:251 602:251 603:251 604:253 605:251 606:251 607:140 608:47 625:67 626:188 627:189 628:188 629:188 630:188 631:188 632:189 633:188 634:109 635:4 +0 126:32 127:202 128:255 129:253 130:253 131:175 132:21 152:84 153:144 154:190 155:251 156:253 157:251 158:251 159:251 160:174 176:6 177:37 178:166 179:218 180:236 181:251 182:251 183:251 184:253 185:251 186:251 187:251 188:251 189:156 204:115 205:251 206:251 207:253 208:251 209:251 210:251 211:251 212:253 213:251 214:251 215:251 216:251 217:180 231:105 232:241 233:251 234:251 235:253 236:251 237:251 238:251 239:122 240:72 241:71 242:71 243:148 244:251 245:180 258:73 259:253 260:253 261:253 262:253 263:202 264:253 265:253 266:143 286:31 287:189 288:251 289:251 290:251 291:31 292:189 293:251 294:142 314:63 315:236 316:251 317:251 318:96 320:124 321:246 322:142 330:21 331:166 332:21 342:73 343:251 344:251 345:251 346:71 349:217 350:142 357:32 358:190 359:251 360:142 370:73 371:251 372:251 373:251 374:71 377:217 378:142 385:73 386:251 387:251 388:142 398:73 399:253 400:253 401:253 402:72 405:156 406:103 413:73 414:253 415:253 416:253 417:72 426:73 427:251 428:251 429:251 430:174 441:73 442:251 443:251 444:251 445:71 454:73 455:251 456:251 457:251 458:251 469:73 470:251 471:251 472:251 473:71 482:42 483:205 484:251 485:251 486:251 487:79 497:73 498:251 499:251 500:251 501:71 511:41 512:226 513:251 514:251 515:232 516:77 525:73 526:251 527:251 528:251 529:71 540:166 541:253 542:253 543:255 544:253 545:227 546:73 547:21 553:125 554:253 555:253 556:143 568:16 569:169 570:251 571:253 572:251 573:251 574:251 575:174 576:105 579:63 580:144 581:253 582:251 583:251 584:142 597:15 598:35 599:253 600:251 601:251 602:251 603:251 604:243 605:217 606:217 607:231 608:251 609:253 610:251 611:220 612:20 627:143 628:142 629:236 630:251 631:251 632:253 633:251 634:251 635:251 636:251 637:253 638:251 639:137 657:61 658:71 659:200 660:253 661:251 662:251 663:251 664:251 665:201 666:71 667:10 +1 130:218 131:170 132:108 157:32 158:227 159:252 160:232 185:129 186:252 187:252 188:252 212:1 213:253 214:252 215:252 216:168 240:144 241:253 242:252 243:236 244:62 268:144 269:253 270:252 271:215 296:144 297:253 298:252 299:112 323:21 324:206 325:253 326:252 327:71 351:99 352:253 353:255 354:119 378:63 379:242 380:252 381:253 382:35 406:94 407:252 408:252 409:154 410:10 433:145 434:237 435:252 436:252 461:255 462:253 463:253 464:108 487:11 488:155 489:253 490:252 491:179 492:15 514:11 515:150 516:252 517:253 518:200 519:20 542:73 543:252 544:252 545:253 546:97 569:47 570:233 571:253 572:253 596:1 597:149 598:252 599:252 600:252 624:1 625:252 626:252 627:246 628:132 652:1 653:169 654:252 655:132 +1 130:116 131:255 132:123 157:29 158:213 159:253 160:122 185:189 186:253 187:253 188:122 213:189 214:253 215:253 216:122 241:189 242:253 243:253 244:122 267:2 268:114 269:243 270:253 271:186 272:19 295:100 296:253 297:253 298:253 299:48 323:172 324:253 325:253 326:253 327:48 351:172 352:253 353:253 354:182 355:19 378:133 379:251 380:253 381:175 382:4 405:107 406:251 407:253 408:253 409:65 432:26 433:194 434:253 435:253 436:214 437:40 459:105 460:205 461:253 462:253 463:125 464:40 487:139 488:253 489:253 490:253 491:81 514:41 515:231 516:253 517:253 518:159 519:16 541:65 542:155 543:253 544:253 545:172 546:4 569:124 570:253 571:253 572:253 573:98 597:124 598:253 599:253 600:214 601:41 624:22 625:207 626:253 627:253 628:139 653:124 654:253 655:162 656:9 diff --git a/data/mllib/sample_linear_regression_data.txt b/data/mllib/sample_linear_regression_data.txt new file mode 100755 index 0000000000000..9aaaa4297548b --- /dev/null +++ b/data/mllib/sample_linear_regression_data.txt @@ -0,0 +1,501 @@ +-9.490009878824548 1:0.4551273600657362 2:0.36644694351969087 3:-0.38256108933468047 4:-0.4458430198517267 5:0.33109790358914726 6:0.8067445293443565 7:-0.2624341731773887 8:-0.44850386111659524 9:-0.07269284838169332 10:0.5658035575800715 +0.2577820163584905 1:0.8386555657374337 2:-0.1270180511534269 3:0.499812362510895 4:-0.22686625128130267 5:-0.6452430441812433 6:0.18869982177936828 7:-0.5804648622673358 8:0.651931743775642 9:-0.6555641246242951 10:0.17485476357259122 +-4.438869807456516 1:0.5025608135349202 2:0.14208069682973434 3:0.16004976900412138 4:0.505019897181302 5:-0.9371635223468384 6:-0.2841601610457427 7:0.6355938616712786 8:-0.1646249064941625 9:0.9480713629917628 10:0.42681251564645817 +-19.782762789614537 1:-0.0388509668871313 2:-0.4166870051763918 3:0.8997202693189332 4:0.6409836467726933 5:0.273289095712564 6:-0.26175701211620517 7:-0.2794902492677298 8:-0.1306778297187794 9:-0.08536581111046115 10:-0.05462315824828923 +-7.966593841555266 1:-0.06195495876886281 2:0.6546448480299902 3:-0.6979368909424835 4:0.6677324708883314 5:-0.07938725467767771 6:-0.43885601665437957 7:-0.608071585153688 8:-0.6414531182501653 9:0.7313735926547045 10:-0.026818676347611925 +-7.896274316726144 1:-0.15805658673794265 2:0.26573958270655806 3:0.3997172901343442 4:-0.3693430998846541 5:0.14324061105995334 6:-0.25797542063247825 7:0.7436291919296774 8:0.6114618853239959 9:0.2324273700703574 10:-0.25128128782199144 +-8.464803554195287 1:0.39449745853945895 2:0.817229160415142 3:-0.6077058562362969 4:0.6182496334554788 5:0.2558665508269453 6:-0.07320145794330979 7:-0.38884168866510227 8:0.07981886851873865 9:0.27022202891277614 10:-0.7474843534024693 +2.1214592666251364 1:-0.005346215048158909 2:-0.9453716674280683 3:-0.9270309666195007 4:-0.032312290091389695 5:0.31010676221964206 6:-0.20846743965751569 7:0.8803449313707621 8:-0.23077831216541722 9:0.29246395759528565 10:0.5409312755478819 +1.0720117616524107 1:0.7880855916368177 2:0.19767407429003536 3:0.9520689432368168 4:-0.845829774129496 5:0.5502413918543512 6:-0.44235539500246457 7:0.7984106594591154 8:-0.2523277127589152 9:-0.1373808897290778 10:-0.3353514432305029 +-13.772441561702871 1:-0.3697050572653644 2:-0.11452811582755928 3:-0.807098168238352 4:0.4903066124307711 5:-0.6582805242342049 6:0.6107814398427647 7:-0.7204208094262783 8:-0.8141063661170889 9:-0.9459402662357332 10:0.09666938346350307 +-5.082010756207233 1:-0.43560342773870375 2:0.9349906440170221 3:0.8090021580031235 4:-0.3121157071110545 5:-0.9718883630945336 6:0.6191882496201251 7:0.0429886073795116 8:0.670311110015402 9:0.16692329718223786 10:0.37649213869502973 +7.887786536531237 1:0.11276440263810383 2:-0.7684997525607482 3:0.1770172737885798 4:0.7902845707138706 5:0.2529503304079441 6:-0.23483801763662826 7:0.8072501895004851 8:0.6673992021927047 9:-0.4796127376677324 10:0.9244724404994455 +14.323146365332388 1:-0.2049276879687938 2:0.1470694373531216 3:-0.48366999792166787 4:0.643491115907358 5:0.3183669486383729 6:0.22821350958477082 7:-0.023605251086149304 8:-0.2770587742156372 9:0.47596326458377436 10:0.7107229819632654 +-20.057482615789212 1:-0.3205057828114841 2:0.51605972926996 3:0.45215640988181516 4:0.01712446974606241 5:0.5508198371849293 6:-0.2478254241316491 7:0.7256483175955235 8:0.39418662792516 9:-0.6797384914236382 10:0.6001217520150142 +-0.8995693247765151 1:0.4508991072414843 2:0.589749448443134 3:0.6464818311502738 4:0.7005669004769028 5:0.9699584106930381 6:-0.7417466269908464 7:0.22818964839784495 8:0.08574936236270037 9:-0.6945765138377225 10:0.06915201979238828 +-19.16829262296376 1:0.09798746565879424 2:-0.34288007110901964 3:0.440249350802451 4:-0.22440768392359534 5:-0.9695067570891225 6:-0.7942032659310758 7:-0.792286205517398 8:-0.6535487038528798 9:0.7952676470618951 10:-0.1622831617066689 +5.601801561245534 1:0.6949189734965766 2:-0.32697929564739403 3:-0.15359663581829275 4:-0.8951865090520432 5:0.2057889391931318 6:-0.6676656789571533 7:-0.03553655732400762 8:0.14550349954571096 9:0.034600542078191854 10:0.4223352065067103 +-3.2256352187273354 1:0.35278245969741096 2:0.7022211035026023 3:0.5686638754605697 4:-0.4202155290448111 5:-0.26102723928249216 6:0.010688215941416779 7:-0.4311544807877927 8:0.9500151672991208 9:0.14380635780710693 10:-0.7549354840975826 +1.5299675726687754 1:-0.13079299081883855 2:0.0983382230287082 3:0.15347083875928424 4:0.45507300685816965 5:0.1921083467305864 6:0.6361110540492223 7:0.7675261182370992 8:-0.2543488202081907 9:0.2927051050236915 10:0.680182444769418 +-0.250102447941961 1:-0.8062832278617296 2:0.8266289890474885 3:0.22684501241708888 4:0.1726291966578266 5:-0.6778773666126594 6:0.9993906921393696 7:0.1789490173139363 8:0.5584053824232391 9:0.03495894704368174 10:-0.8505720014852347 +12.792267926563595 1:-0.008461200645088818 2:-0.648273596036564 3:-0.005334477339629995 4:0.3781469006858833 5:0.30565234666790686 6:-0.2822867492866177 7:0.10175120738413801 8:0.5342432888482425 9:0.05146513075475534 10:-0.6459729964194652 +6.082192787194888 1:0.42519013450094767 2:0.09441503345243984 3:-0.07898439043103522 4:-0.32207498048636474 5:-0.9180071861219266 6:0.5951317320731633 7:0.41000814588717693 8:-0.3926260640533046 9:0.2789036768568971 10:0.13163692286014528 +-7.481405271455238 1:0.03324842612749346 2:0.07055844751995122 3:-0.47199515597021113 4:-0.682690342465275 5:0.3983414713797069 6:-0.2136729393256811 7:-0.09066563475481249 8:-0.4640338194317184 9:-0.03513782089224482 10:-0.1711809802758364 +6.739533816100517 1:0.1774546460228057 2:-0.6783644553523549 3:-0.47871398278230504 4:0.02272121490463097 5:-0.5047649289302389 6:0.26479596144873896 7:-0.32045436544054096 8:0.3113047940487379 9:0.6269418147567556 10:0.9710114516962312 +3.780807062175497 1:0.01715676997104909 2:0.8975962429865936 3:-0.46594560920034134 4:0.2873623499953055 5:0.8894362304584083 6:0.17973981232418468 7:0.49105791400707743 8:-0.7359842740294882 9:0.38941133808001127 10:-0.7151884777228046 +4.564039393483412 1:0.07478785545033317 2:-0.8672651994084235 3:0.450599300176334 4:0.35104802298560056 5:0.6797318185095045 6:-0.03891997518827006 7:-0.33208695871398675 8:0.6166574577055226 9:0.5730212324012205 10:-0.4194925751047054 +-0.3195679646035633 1:0.054527683864544096 2:-0.15591931640565093 3:0.9266742559542833 4:0.888522581905147 5:0.6576203900699167 6:0.6417770212400336 7:0.7509788029052338 8:-0.3104974571382815 9:0.7234744267051683 10:-0.15869049651427103 +11.290452658023497 1:0.20173310976772196 2:0.8657502566551409 3:0.9325160601080682 4:0.24570884032596263 5:-0.6546108813337841 6:-0.14020032028377583 7:-0.8825687891702743 8:-0.21420166926412865 9:-0.8600275184792756 10:-0.7990574622230739 +-4.003499192090455 1:0.8325875503351796 2:-0.5956350140619129 3:0.12598048009007923 4:0.12340188733473134 5:-0.839435659309717 6:-0.16623481818728414 7:0.12028795301041662 8:-0.7994713170657952 9:0.2216721974907896 10:0.8407561415075087 +-19.872991038068406 1:-0.9325810772922609 2:-0.6411471147334535 3:0.9949216290375054 4:0.483048267470493 5:-0.8736297429070232 6:-0.36222771685582544 7:0.26397860162786957 8:0.45527588775737704 9:-0.9424989711186325 10:0.6251162293059616 +10.502762149373098 1:-0.2307778924009991 2:0.6977871128979924 3:0.022830408261390822 4:0.6257738824362347 5:0.9770979848265122 6:0.09985730624684575 7:-0.9755858424230182 8:-0.689969833240031 9:-0.7294587311376761 10:0.3496326193951331 +-14.328978509075442 1:0.37929821892417404 2:0.8402056881660709 3:-0.1806835799958202 4:0.766314307210441 5:0.865876513623024 6:-0.7113501219432434 7:-0.0932956557986735 8:-0.7042025810921411 9:0.47530696925672267 10:-0.4629102077669889 +-16.26143027545273 1:-0.9309578475799722 2:0.7591795880911123 3:0.06296957473213705 4:0.786790093290086 5:-0.9527998391625465 6:-0.08573982501921895 7:-0.3812232026687308 8:-0.6890669703685022 9:0.25415911467755015 10:-0.07664746267502509 +11.772544195529013 1:0.3614756404325046 2:0.14508027508253818 3:0.23042774014795753 4:0.4164348685332022 5:0.4109091750657461 6:0.03853098236933272 7:0.38911994885223145 8:-0.5031309357181766 9:-0.596467768575587 10:0.17884522225228028 +14.697703557439503 1:0.24508864174863 2:0.7576193329655578 3:0.09030511120334461 4:0.9537528991778741 5:-0.7224092160621338 6:-0.34089385162121943 7:0.6924170720838818 8:0.32912306214891784 9:-0.4064624712125904 10:-0.5344662061201593 +-13.976130931152703 1:0.5891192531479754 2:0.29862103742464274 3:-0.36153976712796343 4:-0.6552669564323226 5:-0.22672513691161766 6:0.3001336202535376 7:0.34490251346382617 8:0.2072633053920192 9:-0.5659371284058774 10:0.49599636156628835 +-14.762758252931127 1:0.31302496164254223 2:-0.6062773982342133 3:-0.9874007658402217 4:-0.6214904627601421 5:-0.11421073677207683 6:-0.5850843421161205 7:0.1250679146774638 8:-0.7108170726393621 9:-0.6888351241194393 10:0.6077343683084389 +-3.300641320608255 1:-0.1407178879203672 2:0.12960233233004925 3:-0.4236196478321872 4:0.7903078296084356 5:-0.8755754953628643 6:-0.2062360260394529 7:-0.045680124889026175 8:0.783182093429277 9:-0.02995737262668463 10:-0.33382351650328435 +-15.72351561304857 1:-0.1802575775708093 2:-0.991006951265341 3:-0.9107951763247621 4:0.9069820084047908 5:-0.12691921206803047 6:-0.7087012119383593 7:-0.9179510577925369 8:0.18480349982718325 9:-0.4478459144114004 10:-0.5560585660624608 +-22.949825936196074 1:0.4797855980916854 2:0.01997502546020402 3:-0.8827928315487465 4:0.2755107907750989 5:0.015544482147298977 6:0.9652687138748801 7:0.6622667860970648 8:-0.7708138539912186 9:0.17728148663006627 10:0.47818190728952925 +12.092431628826905 1:0.1358843437335564 2:0.03643446587894239 3:-0.31070823939673287 4:0.5283033206569152 5:0.3469111543845367 6:-0.5162518174930761 7:0.24270234207184016 8:0.7352292800096338 9:0.8860322286740037 10:0.6748068653962045 +-23.51088409032297 1:-0.4683538422180036 2:0.1469540185936138 3:0.9113612952591796 4:-0.9838482669789823 5:0.4506466371133697 6:0.6456121712599778 7:0.8264783725578371 8:0.562664168655115 9:-0.8299281852090683 10:0.40690300256653256 +5.998186124881712 1:-0.9781302074883151 2:0.32984303335155785 3:0.7303430847899663 4:0.841481297188956 5:0.05580773881989276 6:0.7130788298702062 7:-0.218087116119847 8:-0.9889494995220598 9:0.9182854134226501 10:-0.7501751701020942 +9.852316338642547 1:0.146854160091757 2:-0.3611508707370965 3:0.3517016971654914 4:0.6187697988029395 5:-0.010768583697787548 6:0.5236725885871243 7:0.5945666964145524 8:-0.009180562740628506 9:-0.44474762415618274 10:0.41852743519493685 +-5.313930756588526 1:-0.6304209277071555 2:-0.37010359785263813 3:-0.3194739026510125 4:-0.750533359080716 5:0.45500303301733114 6:-0.012727544364283805 7:-0.43941651856862274 8:0.927108876532093 9:-0.24164903158058149 10:0.44134972919002124 +-4.2775224863223915 1:-0.35785764991284363 2:0.942797043714243 3:0.4539569191274251 4:-0.6944903010994341 5:-0.08357221983075225 6:0.4433049548665855 7:-0.5488972050023557 8:-0.24014623658145773 9:-0.6178118485382511 10:-0.4575463952834564 +-10.57769830424322 1:0.22693864400257335 2:-0.041639691095668674 3:0.9948726461115123 4:-0.7450471554938383 5:-0.1114847126717804 6:-0.27881184842402673 7:0.029766812446276214 8:-0.3727649352432578 9:-0.7791732805568077 10:0.9425576681069683 +-0.8430338600258201 1:0.4607090007225536 2:-0.6079961642969514 3:-0.5671626932935381 4:0.12784576080614185 5:-0.30766031989910236 6:-0.21232963505711555 7:0.3310463755850872 8:-0.6807682731528943 9:0.7826634145951483 10:0.0608057623636995 +13.450586257053727 1:-0.2697769964284986 2:0.07743737732312428 3:-0.8459687499864881 4:0.6091901514177853 5:-0.9464815428211699 6:0.15780407422581533 7:-0.28552052619478996 8:-0.27500859181806403 9:-0.7207541548282903 10:0.05215593729084533 +20.358241877831016 1:0.29768927445620164 2:-0.5379390525163252 3:0.6591913001003027 4:0.6635992348010928 5:0.3786594651413009 6:-0.7217135278882543 7:0.9634013908615768 8:0.03961253903778861 9:0.1335121312144949 10:0.7933944303463509 +9.800993960518852 1:0.39896823489212285 2:0.30948413101894023 3:0.08568060094378493 4:-0.7454513450113371 5:0.8054125831421357 6:-0.24464240413169347 7:-0.18294406588625112 8:-0.883455504399858 9:0.2468431033653562 10:-0.708151566382103 +-21.432387764165806 1:-0.4785033857256795 2:0.520350718059089 3:-0.2988515012130126 4:-0.46260150057299754 5:0.5394344995663083 6:0.39320468081626836 7:0.1890560923345248 8:0.13123799325264507 9:0.43613839380760355 10:0.39541998419731494 +-4.090570760187878 1:0.3909705814857716 2:0.9830271975811611 3:0.672523651785939 4:0.0035177223850744177 5:0.567082732451311 6:-0.2620454326881394 7:0.46622578556708105 8:0.646246879249865 9:0.4263175536668733 10:0.8982696975276223 +3.7459201216906926 1:-0.9480167656870653 2:-4.888270196095057E-4 3:0.48226844071577646 4:-0.23706663537631645 5:0.22420266627462127 6:0.2981747607694978 7:0.3893425967975348 8:0.6302701381298614 9:-0.21909113816064196 10:0.8371697958140494 +9.767952084958061 1:-0.2300790371078303 2:-0.4457883630748676 3:0.28710853302295325 4:0.7112839743052013 5:-0.8765858382640623 6:-0.6470779468607217 7:0.4369262584371727 8:-0.7175412028407337 9:0.5506733477278882 10:0.5393007189573547 +6.9802839308913365 1:0.21769855012808215 2:0.8653818331675485 3:0.2322943113578111 4:0.3760591265797468 5:0.06554014167292377 6:0.6866096712933549 7:0.866929973115441 8:-0.6462263417217329 9:0.2507247465275353 10:-0.7005877782050307 +16.014720800069103 1:0.6058055248984549 2:0.048517868234337014 3:-0.15744912875924877 4:0.32598079708869365 5:-0.587791997223768 6:-0.4636187312118474 7:0.7771908559246068 8:-0.349403853888719 9:0.229800030145503 10:-0.674614818934488 +8.417571532985823 1:-0.21164946152466801 2:-0.9981936663594053 3:0.8611869575187896 4:0.11100891297254312 5:-0.7406067304729631 6:-0.7613837395522254 7:-0.9617573325708704 8:0.5697426971647488 9:-0.5830879716990833 10:0.5951448538064159 +-12.491442077546413 1:-0.19172117564625735 2:-0.12421304883392126 3:0.7095605786791346 4:0.6401582292398038 5:-0.9347790209840108 6:0.6592209285686903 7:0.702282297844389 8:-0.22765902007749528 9:-0.17746922342943816 10:0.7196663432778121 +-8.605713514762092 1:0.36490454976480846 2:0.6991204480538957 3:0.6546945560337121 4:-0.032324845758738174 5:0.2453935969836043 6:0.5363119225093116 7:0.6266741350524205 8:-0.2132266305382322 9:-0.308105870487996 10:-0.08219413867616465 +-10.35591860037468 1:-0.014204168485027147 2:-0.7077035677144325 3:0.024004217785642767 4:0.818971992516166 5:0.9081305263471056 6:0.808854493237229 7:-0.6474336785461867 8:-0.32559288177031465 9:-0.32850453072496055 10:-0.7035310416695784 +3.605002621628445 1:0.6085817977516599 2:0.8101072412357928 3:0.7697891508923966 4:-0.5738750389864677 5:-0.734314989863889 6:-0.7879014492215499 7:0.6884442838920775 8:-0.46131231930402383 9:-0.7730585954271005 10:-0.7819874019145132 +12.30435312415091 1:0.3283668768730639 2:-0.18316686990068187 3:0.3955614099142126 4:0.8450470350842108 5:0.3958042901611589 6:0.6578475571960676 7:-0.4395488558075096 8:0.15720430113495376 9:-0.5318362828977672 10:0.45585285255232044 +9.020048819638827 1:-0.5986521145193395 2:0.3266542215286443 3:-0.09911773729611917 4:-0.21478254478908676 5:0.6546175049764293 6:-0.1414796368932345 7:0.25802631337510085 8:-0.6773828562539816 9:-0.22038193899258718 10:-0.17465737306657902 +14.854262978981406 1:0.5293763924477841 2:-0.24658868331583683 3:0.8268631648872109 4:0.8969207203400265 5:0.03933229861213983 6:-0.6212951181360529 7:-0.36695460282178205 8:-0.5468014636386027 9:-0.3419492829414976 10:-0.8273314086998671 +5.658665647926016 1:0.9543096383762801 2:0.13230023957687176 3:-0.3071929861496465 4:-0.3646067841449696 5:0.6979929890816723 6:-0.20721664168809228 7:0.6676482547655365 8:0.944757051233543 9:0.024377296173674567 10:-0.9413728609667691 +-6.930603551528371 1:0.09198647857985232 2:-0.3685113649452161 3:-0.2361728930325453 4:0.3674268130607439 5:0.27385598384498344 6:-0.7151900241735676 7:0.3310154476154119 8:-0.24328111897361682 9:0.2511378679668912 10:-0.35825141175578934 +13.361196783041926 1:0.11676665169094824 2:-0.49968608916548307 3:0.9941342810313298 4:-0.17858967215374988 5:0.1993744673440312 6:0.14596837574280297 7:-0.8245495433125194 8:-0.5637934691545672 9:-0.8589185806222286 10:-0.4923216901915597 +-3.280508467210429 1:-0.9917770074538397 2:-0.1547651813493751 3:0.621733177563484 4:0.7303326279246298 5:-0.0786900332560696 6:0.9107127797641994 7:0.7104513024299466 8:-0.32858522942354407 9:0.17013652749847386 10:0.27656984316288824 +11.13509519160867 1:0.6874932143640391 2:-0.46610293161038907 3:0.8744681017967024 4:0.40900365224695956 5:-0.49770054448432055 6:-0.0635770754462921 7:-0.5705387648707747 8:-0.577988250149829 9:-0.8099463063934682 10:0.42132700180827354 +-11.857350365429426 1:-0.24607974991258308 2:-0.943388538022258 3:0.8679112109377674 4:0.7779951176637694 5:-0.5802336023276593 6:-0.9093352471884992 7:0.29337797938742316 8:0.498519874589175 9:0.3493034812120912 10:-0.07307210651399076 +11.421632138263703 1:0.3911519359353859 2:-0.8154393787235621 3:0.47194271125243237 4:0.14014792298759593 5:-0.3589345913619957 6:0.7887695409762479 7:0.49962792312858895 8:-0.6402670146359797 9:-0.2314041601683119 10:-0.798901341175887 +5.194792012146463 1:0.810279303469398 2:-0.9772756877199589 3:-0.20925958437085557 4:0.8797562461102444 5:0.3211532423260066 6:0.25250279470783754 7:0.14387831263435813 8:-0.021466789385169882 9:0.18909293657271564 10:-0.5981349964027893 +12.242677118499806 1:0.3565715672082048 2:0.7366743237221687 3:0.1922233582434527 4:-0.3551925780624561 5:0.5290849503909634 6:0.7744214641246749 7:0.7277215028580597 8:-0.590440215391044 9:0.7427328184290733 10:-0.6231904162251609 +3.496172341296411 1:0.5028717258135624 2:-0.5838871888624848 3:-0.5540116561110324 4:0.8502487679795261 5:-0.7983061034328727 6:-0.3853123296389005 7:-0.1493800684643869 8:0.6008798629354264 9:-0.32299062155495406 10:-0.5827019502242026 +-15.437384793431217 1:0.41994681418237345 2:0.7106426870657483 3:-0.45211033467567696 4:-0.7272406549392239 5:-0.35736594496490737 6:0.4764507578985955 7:-0.5249912641281373 8:0.8562010912051132 9:0.45927621623833637 10:-0.3701817429794385 +5.490036861541498 1:0.8414999442459015 2:0.9273442862476728 3:-0.054654787893199774 4:-0.23126134156257327 5:-0.9155048245317694 6:0.25750538376376975 7:-0.8470916763665326 8:0.9105674676753848 9:0.5026028522378054 10:-0.06650501561108468 +-1.074065343287859 1:0.37484830603001607 2:-0.9858854245832975 3:0.007159356555897611 4:0.8172796295244154 5:0.519147377529164 6:0.8211049991970722 7:0.9901658817979146 8:-0.026519560032641998 9:-0.2328762488733862 10:0.43161994187258035 +2.0482082496444622 1:0.24940246021565793 2:0.47248358864259177 3:0.23833814894291105 4:-0.3449172512379757 5:0.7412869866239866 6:0.1351422898741914 7:-0.003784141556894216 8:-0.6321917152754075 9:0.8246267827865776 10:0.5057520480449009 +16.709794859608397 1:-0.5977424405191092 2:-0.13991362149785713 3:0.613487896720806 4:-0.37681525320218157 5:-0.4369592282569783 6:0.4702242879506955 7:0.07498463532645339 8:-0.9942304127133292 9:0.41304209196175257 10:0.6799250665519481 +4.598881854940949 1:-0.41212838137243835 2:0.6737124633791323 3:0.8376369191216593 4:0.2848328781926128 5:-0.17960265353296 6:0.0035488712665472377 7:-0.8355355482928055 8:-0.7439716673142398 9:-0.009043467128117433 10:0.7423272515054122 +9.566038608555402 1:-0.662329643040616 2:0.4727113884417973 3:-0.15734218732411365 4:-0.3950754785173889 5:0.13837083076070011 6:0.633261314089351 7:0.9929998062307679 8:-0.4639028424346423 9:-0.073992579817449 10:0.3413166410117088 +1.629198477883475 1:-0.2875719791707101 2:0.9395753700232541 3:-0.45090801750966314 4:-0.384528069378699 5:-0.35937736478702753 6:0.9597102694501136 7:-0.6898325123180971 8:-0.11436012866371303 9:-0.5330550575952768 10:0.24688769932037258 +-7.374620970147229 1:0.16864051681940984 2:-0.08391828256018252 3:-0.8184503043836224 4:0.5461252511055263 5:0.7264676659099087 6:-0.9654384426822686 7:-0.8537533138667612 8:0.9189716013058653 9:-0.03449322582531389 10:0.5490329745887035 +-0.5741704240890674 1:0.9392753294760656 2:-0.5579682000156501 3:-0.8083270703362093 4:-0.7022804026958895 5:-0.30426803430649896 6:0.8211432527140852 7:-0.8101343265051797 8:-0.0945946325760949 9:0.49546915718101814 10:0.5184327698839013 +12.583032451116004 1:0.20496323995364651 2:0.5082017540304999 3:0.2428646053751764 4:0.7101854338863274 5:-0.9619925264660094 6:0.4610134502825909 7:-0.5620669052678122 8:0.6766614078376236 9:-0.7169693435782278 10:-0.14362322382035164 +-10.489157123372898 1:-0.7441633083637054 2:0.07069898351187809 3:-0.47119552972566336 4:-0.43970155900871344 5:0.43192289605353973 6:-0.0798550143899397 7:0.2111188135787776 8:0.9101748615761336 9:-0.4079984876629721 10:-0.8101424982394589 +-3.811365493249739 1:0.7250263461647963 2:0.22182621035333838 3:-0.12735342714215725 4:0.26222861719040624 5:0.3928174057935714 6:0.817131411734006 7:-0.056109765698795 8:0.7908779197353637 9:-0.06768319505245768 10:0.4107045608924882 +-7.604636483513961 1:0.876751634787073 2:0.04037085575852295 3:0.18142385658771398 4:0.38350565074271903 5:-0.30937664332011905 6:-0.9544807672006823 7:0.008643477632712449 8:-0.27676843472226276 9:-0.12938540988602476 10:-0.2929762262661819 +-1.9889499615051784 1:-0.4243149295090465 2:0.22578711943818686 3:0.662530786460152 4:0.28592235843136105 5:0.4170345231441832 6:0.9290881132120887 7:0.5332443368002588 8:-0.33248958421809927 9:0.16273139830495942 10:0.6899022585936985 +-1.99891354174786 1:-0.1732078452611825 2:0.2361029542296429 3:-0.8455867017505336 4:0.31638672033240867 5:-0.648387667144986 6:-0.7647886103837449 7:0.6910155501192978 8:-0.2665663102538198 9:-0.5980899570876459 10:-0.9165896495676276 +9.74348630903265 1:0.18934450539532244 2:-0.715110505416745 3:-0.453777527810155 4:0.2743741252197758 5:-0.8439310405443103 6:-0.533835190276116 7:-0.5911710854054728 8:0.21026462628920695 9:-0.45849607678093585 10:0.1327074179200407 +20.221961806051706 1:0.624731930687735 2:-0.39914395421723015 3:0.781887900750925 4:0.5442619051596436 5:0.16651193067479153 6:0.9064846121246533 7:-0.3643159594276202 8:-0.5182065337246469 9:-0.6785628247191553 10:0.7111152852903913 +20.456947955410897 1:-0.21923785332346513 2:0.11340668617783778 3:0.7397883986253251 4:-0.11748081084695605 5:0.06314872700777197 6:-0.7124574845946587 7:0.18043581960897104 8:-0.09023925260092103 9:-0.7256417560118238 10:-0.5038088673851804 +12.241006086129564 1:-0.15271598143132215 2:0.9038942665552285 3:-0.6168514099878155 4:-0.12219038322317011 5:0.5402785935596728 6:0.4059744401803913 7:0.258870596734184 8:0.3190881033039108 9:0.2372469007313076 10:0.367188299614863 +3.980473021620311 1:-0.9025895351376971 2:-0.03333947011476446 3:-0.8220776066161464 4:0.449117985679933 5:0.9970519437779266 6:0.27430911004640457 7:0.039081352882204046 8:-0.8621514950929796 9:-0.569587565933642 10:-0.9118346349929578 +-13.420594775890757 1:0.3697979495309094 2:0.07383664120111888 3:0.7199366131785143 4:0.2118625428869032 5:-0.9015976323216077 6:-0.5298395275757712 7:-0.9517419542156635 8:0.39554920787574743 9:-0.3721957439110324 10:-0.4750272836396878 +-1.052659359353786 1:0.02106845330888185 2:0.7571245678782959 3:0.8034228830223251 4:0.32968340513846917 5:-0.6510386482911554 6:0.2710115488605187 7:-0.1319580272290235 8:0.932600992666184 9:0.8260461527035414 10:-0.8507648952138052 +9.813440129324034 1:0.41048687946340134 2:0.9384639988086239 3:0.4569555844323441 4:-0.3084729082645552 5:-0.7299010284877061 6:-0.6925012997779212 7:-0.6798013915257548 8:-0.504368104320321 9:-0.6234398059664716 10:0.8633407902005543 +-2.8942782378157714 1:0.5546381825677706 2:0.7959405841824887 3:0.584699836289184 4:-0.5726371777829862 5:-0.2827976152663936 6:0.138034013875719 7:-0.2935080791661324 8:-0.5323479091625714 9:0.6837641044797451 10:0.5986680812032501 +8.562937733537664 1:0.14753220510180776 2:-0.31591341855048327 3:-0.748545617199091 4:0.3251888821665734 5:0.8228589483149358 6:0.046659706976506676 7:-0.35049927996132624 8:0.2953170004605874 9:-0.6429374177050204 10:0.4624083116836044 +13.413187970975178 1:-0.7229883396779724 2:0.8876940454894067 3:-0.033794226589695775 4:0.46700071356381523 5:0.1599557295166274 6:-0.8944619785248653 7:-0.1258464584151997 8:-0.8797551785991506 9:-0.14408879184669354 10:0.11020655997336015 +-5.491389764900794 1:-0.366507395597937 2:0.630480481240723 3:-0.16600801981741609 4:0.09842042773854076 5:0.30129535029579047 6:0.14102166298628882 7:-0.28131788612036623 8:0.49635295715686234 9:0.0625636989631968 10:-0.41748132718912 +-10.29566593602992 1:-0.7898597726154271 2:-0.05425577320946573 3:0.5992645759265662 4:-0.4716868549309716 5:-0.020137302700854676 6:0.6216515277233232 7:-0.7295510954484412 8:-0.41443875567123967 9:-0.610576632050404 10:-0.9515988311377204 +7.084732852050431 1:0.9990215581592679 2:-0.9868954542412269 3:0.49133473382040704 4:0.7697599878561228 5:-0.34668939907967267 6:0.9777705993519483 7:0.4449043102759509 8:0.9812971199646168 9:0.6666598587737487 10:0.14398842572598514 +0.23715467505851734 1:0.21628799185444336 2:-0.4526390568867018 3:0.6558486691929235 4:0.13730688681492142 5:0.23076986155942736 6:0.7020484017619715 7:-0.12077999528458938 8:0.8306084972447003 9:-0.49337323198621563 10:-0.8270028152572872 +1.1552619549601455 1:-0.48202394020369277 2:-0.6274878708695264 3:-0.27623674153600697 4:-0.5312153415813432 5:-0.030820182786174044 6:-0.5893370965577813 7:0.6666315120904487 8:-0.36482991729570036 9:0.6065771813692735 10:0.05831057330788525 +-0.20433879835596253 1:-0.4702220250018212 2:0.9123705796362889 3:-0.2045657170490376 4:-0.18922063450309534 5:-0.31431213362503163 6:0.4150130060120387 7:0.34016193625941127 8:0.8391374136299805 9:0.6884250315764333 10:-0.7916408854251566 +-9.751622607785082 1:-0.0014232315621649505 2:-0.1284246813729939 3:0.5228953023175369 4:0.9688522449007109 5:-0.7857721219549156 6:-0.7812922263391038 7:-0.5916136652814756 8:0.793988610184206 9:0.7982949061274296 10:-0.592785473963741 +-22.837460416919342 1:-0.17363144173810174 2:-0.3340314573781735 3:0.9351424971322297 4:-0.6430601902397572 5:-0.13363305808148818 6:-0.42446359566938585 7:-0.4093070316761178 8:-0.9302259781839204 9:0.47004365892170585 10:-0.6231289889808045 +-3.6318714209289436 1:-0.8296410705737971 2:-0.6056572341069668 3:-0.2975417404042737 4:0.07134138175064741 5:-0.8966463747179154 6:-0.4112675899658855 7:0.7908013478009401 8:0.407396254566472 9:0.9227769302156879 10:0.12418427404473764 +-3.8909712376010583 1:-0.6552751548581366 2:-0.5641921108932855 3:-0.6340486345063014 4:-0.5441069121131075 5:0.908720622198947 6:-0.026054643814348077 7:0.03706191653058433 8:-0.6672524338819317 9:0.7958274915288801 10:-0.19029619970124023 +-10.600130341909033 1:-0.7457695999520562 2:-0.3739453132549577 3:0.01327423342620393 4:-0.08482897201178563 5:0.84573456086082 6:0.6279927575103963 7:0.014494803555804125 8:0.9420647557771027 9:-0.13484113287285893 10:0.3037405853352888 +-12.094351278535258 1:0.9982796018306028 2:0.8354271779265348 3:0.46284321795736116 4:0.07693347919601745 5:-0.4753440408996932 6:-0.47098252868073787 7:0.4810729184846003 8:-0.6136990339205741 9:-0.6715833036640317 10:-0.6247058955319091 +9.936399360181602 1:0.7330323083522969 2:0.47204204993669197 3:0.3850471475752122 4:0.21483460195167958 5:0.3806220122265147 6:0.6336993433402796 7:-0.47987416364572 8:-0.195509010865196 9:-0.6561820282562041 10:-0.45300480439842894 +-4.706701061062994 1:-0.847895844561626 2:-0.29946646506145114 3:0.25432868082106497 4:0.1352958872054535 5:-0.8803017974303002 6:-0.3675110562764785 7:0.10888496324899721 8:0.9620737605396772 9:-0.031046632561323895 10:-0.09466883461500908 +5.101614991255809 1:-0.5174248135588373 2:0.14394061894828014 3:0.5613709266711013 4:-0.5678634944326011 5:0.930216209978763 6:-0.6204727890080077 7:0.4133141749872311 8:0.6262685035917408 9:0.03382924477926896 10:-0.15231139191832854 +-8.772667465932606 1:-0.2117605577769197 2:-0.4283897136887762 3:0.44686767473401035 4:-0.5507826261358746 5:0.237124956028401 6:0.6413157520982717 7:0.2409214827604571 8:-0.8505503638033114 9:-0.9811997368468401 10:-0.9499963936664035 +-11.615775265015627 1:0.8782018665273386 2:-0.9751473570197167 3:0.6307050068521085 4:0.7012721336851997 5:0.21318736263512283 6:0.024885128053773853 7:-0.4580644243558505 8:0.1318650007251434 9:-0.9306090092992167 10:-0.5688746770986652 +19.64829023536192 1:0.14426537998360645 2:0.3557716894181753 3:-0.8577143134654568 4:0.5288643233801469 5:0.9231529738221469 6:0.975999712077738 7:0.24700404691888678 8:0.10206517527052283 9:-0.10041951294847062 10:-0.9412918491876225 +2.7409415438025486 1:-0.7404936009304737 2:-0.9792071376296605 3:-0.49510748520932113 4:0.9538460112904268 5:-0.5075114153141447 6:-0.5890791308058669 7:-0.775366087491284 8:0.4983912525892249 9:-0.2976197956132913 10:0.6791258030468514 +-4.394658158733604 1:-0.41628618754613345 2:-0.1472602552309057 3:0.44136102233464025 4:0.011882653940414434 5:-0.6559502840386595 6:-0.4022529016339016 7:0.048402312931387526 8:0.8753776623326166 9:-0.8528247288266961 10:0.6593783978826002 +1.1915739133607073 1:-0.7840827624854878 2:-0.4860418508208426 3:-0.7418773161179972 4:0.129874781837924 5:-0.22631682294184796 6:0.47794208013755024 7:0.5532183426143056 8:0.11879859459306741 9:0.09927630694484524 10:-0.19268618891399636 +2.156192215438919 1:0.44325986644475646 2:-0.6057278708888592 3:0.3943381582091723 4:0.6560336238050575 5:-0.9651308100517204 6:-0.2358219003943678 7:-0.04143043460232465 8:0.8623951169233035 9:-0.4933545255502605 10:0.8990427200454263 +-1.1009750789589774 1:-0.4515707618788496 2:-0.745936099912899 3:0.41307003181926794 4:0.6127760492402428 5:0.9250878169732681 6:-0.6778628527469126 7:0.42794190420905753 8:0.4943969797578971 9:0.7762709104958854 10:-0.6932349268610041 +10.04434496594037 1:-0.0995467494040092 2:-0.7766769414838959 3:-0.6608009972582911 4:0.7075788021090594 5:0.5208396359138381 6:-0.09724033794207299 7:-0.743087245352148 8:0.765372791789753 9:0.3788699859744704 10:-0.6898257995055466 +8.038039859115667 1:-0.5321510657026671 2:0.5571925538006008 3:0.747268102801854 4:0.09079641165917596 5:0.25861122989509266 6:-0.9948187479498878 7:-0.9665136866462685 8:-0.3904629432867681 9:-0.9975425877998279 10:0.32024289816988416 +5.14371929922303 1:-0.4829199170694627 2:-0.5713285263827719 3:-0.43889652467111184 4:0.18478247261988967 5:-0.27374063120041225 6:-0.8069125377696931 7:-0.15497746743367058 8:0.32448521325998714 9:-0.39397735035206227 10:0.08184957956614292 +-1.6848276484379352 1:-0.39250489761445895 2:0.02730338852529557 3:0.9916055514435305 4:-0.07571433435055064 5:0.19024527726403728 6:0.6385182319185971 7:0.32480605537471297 8:0.5807543325220577 9:-0.35642510103381153 10:-0.9060482769392468 +-11.640549677888826 1:0.03707410390488852 2:0.2527049166981137 3:0.4114872952854447 4:-0.8508977901757795 5:-0.42791544663481895 6:-0.9864047295390463 7:0.6023685964407528 8:0.12018443688097036 9:-0.36816249877130414 10:-0.9583147535652901 +11.672104494601319 1:-0.2416258355340175 2:0.6737553249072334 3:0.9041602191361382 4:-0.2123232797997281 5:-0.008255188002961988 6:-0.5151894064136904 7:-0.7341877977528246 8:0.624625272218277 9:-0.6261434804192929 10:-0.5710586715741532 +-2.2960192492344627 1:-0.7457768645184579 2:-0.5954998103421847 3:0.5428846769211537 4:-0.6176587961491775 5:0.46222150678166574 6:0.7852238239427731 7:-0.3614580530629148 8:-0.325840253127059 9:-0.5660596710348922 10:-0.8060263366626401 +5.428302298615722 1:0.8774286357993033 2:-0.23110126319781088 3:0.6264134914476072 4:-0.143015582616014 5:0.350109539755298 6:-0.147747167834422 7:0.05020570422182824 8:-0.5469605849960337 9:0.951112567977048 10:-0.34800121380288185 +-17.32672073267595 1:0.31374599099683476 2:-0.36270498808879115 3:0.7456203273799138 4:0.046239858938568856 5:-0.030136501929084014 6:-0.06596637210739509 7:-0.46829487815816484 8:-0.2054839116368734 9:-0.7006480295111763 10:-0.6886047709544985 +7.314490512652487 1:0.8745354279105222 2:-0.9270067504840309 3:0.965218170323435 4:0.12808957052353698 5:-0.5309399625085234 6:-0.5968520990090951 7:-0.667403236513185 8:0.08154410986660832 9:0.33025488397543934 10:0.03406708067839537 +4.687373993408297 1:0.6731426721418288 2:-0.7111023070261273 3:-0.9849054116048603 4:-0.12831346258317322 5:-0.04095946352836921 6:0.6967001556166801 7:0.8479895229743999 8:-0.35600791972899404 9:0.5005979045264868 10:0.6421341979636503 +-6.82923852156868 1:-0.04849233571020073 2:-0.8505855619911602 3:0.2927180954190314 4:0.5780268040086791 5:-0.22322207765417268 6:-0.8436513934568071 7:-0.3906240514635124 8:0.7258714963093444 9:-0.21695043530813085 10:0.8049335285918169 +-8.24622879369294 1:0.12154833675098842 2:-0.26446415445316673 3:-0.06653791221669247 4:-0.7920694887292259 5:0.6128791496627621 6:-0.6927179137980173 7:-0.24584418172709932 8:0.3557416365779935 9:0.22868636757755234 10:-0.8288196322549064 +-5.090863544403131 1:-0.1535668648046895 2:-0.59868738365189 3:-0.8822518703008675 4:-0.1790505106198006 5:0.9467581256591948 6:-0.0661313762905984 7:0.31263046332923694 8:-0.03628894224569357 9:0.8969599435828515 10:-0.05386674051170348 +-15.780685032623301 1:-0.2568492063716883 2:0.7740976197426315 3:-0.7829158104387535 4:0.8578846037465748 5:-0.6111039318672586 6:-0.26939268282639306 7:0.3659136640533909 8:-0.8205938562638555 9:-0.24945505706767923 10:-0.935948184861368 +-3.9916779937384743 1:0.22925954469403154 2:0.18159238246979537 3:0.05607027262862396 4:-0.3376037702047998 5:-0.10630000583678934 6:-0.7148277241201622 7:-0.08327294541727137 8:0.6532439360618307 9:0.34352364313237294 10:-0.21028242388807916 +8.798748248458631 1:0.509058184822212 2:-0.17532831457577935 3:-0.6387880909085213 4:-0.966194650702529 5:0.7829797328120436 6:0.5470735549914605 7:-0.38312745239682333 8:-0.8890923931840893 9:0.6823342859396513 10:0.9231260597729121 +14.341273640964873 1:0.6996156678090684 2:0.8612833977834464 3:0.9131301694042417 4:0.5199385192744859 5:-0.32605907950755086 6:-0.9816465962348846 7:-0.5939885763232406 8:-0.7730924566676425 9:0.6367821449954114 10:0.10873812383881054 +9.75855501262469 1:0.2933324921347933 2:-0.4652534314332506 3:-0.2940640558090537 4:0.9883453215038367 5:-0.042460731786114314 6:-0.15438550895912062 7:-0.11182397625560592 8:0.7425954283250873 9:0.5063859049644963 10:0.3012211854180429 +7.695200921242407 1:0.3554353390157281 2:0.08707592690448718 3:-0.10815435665633877 4:0.05524046679762784 5:0.8000157491787581 6:0.3756193347272323 7:-0.18659830666742527 8:-0.08168623764933125 9:-0.2551379303720174 10:0.8560030587463281 +26.903524792043335 1:-0.4672678144441864 2:0.868381965588082 3:-0.04748335609643428 4:-0.0908285508827269 5:-0.22436865911994275 6:-0.953965287326564 7:0.40644848732968164 8:-0.33391575325981115 9:0.008337907338700212 10:-0.45597904754961416 +9.87318781117539 1:0.7310287890171705 2:-0.38300115058116324 3:0.5492682498036086 4:0.552016070316655 5:0.3715022458396897 6:-0.3888040017277252 7:0.21348231125683648 8:0.23219558685722874 9:-0.6271161253492459 10:-0.009137052604519136 +7.6930514050666625 1:0.48603550488592284 2:-0.9218820771919889 3:0.17678612698428053 4:0.5110501870908806 5:0.5817010201164554 6:0.4488707800038747 7:0.4977618637956498 8:0.1683214570038094 9:0.17237242672259323 10:-0.5276084644007359 +3.155413914311745 1:0.04582517188512947 2:-0.9060800653779759 3:0.049786270132956556 4:-0.4236784487542993 5:0.6293910028372613 6:-0.7370237043436467 7:-0.4599678991281728 8:0.5317111095323057 9:0.0029525239228334055 10:0.9294876800738165 +-10.18815737519111 1:-0.9023553189306839 2:0.08434165073970856 3:0.7163931103395633 4:0.41749986495957914 5:-0.8190972970472759 6:-0.9996126872234177 7:0.1779075727741255 8:0.18212754689351862 9:0.24628508239298963 10:0.667589863190412 +18.585731475373457 1:-0.8399129036462931 2:-0.10024819268489127 3:-0.5011350892733817 4:-0.7299256348863585 5:-0.9412022985072928 6:-0.245064895931544 7:-0.1032512650854267 8:0.9943863256441088 9:-0.6429371028855466 10:0.062299742931960056 +8.998359297106072 1:-0.16850226855111905 2:0.7262839202089402 3:-0.04876255055071854 4:0.8948164957242868 5:-0.10720585418953132 6:0.2622719447841948 7:0.26433837506661373 8:-0.5143449147399106 9:0.17444585508955002 10:-0.813182163328944 +13.032424230011074 1:0.4014766166181287 2:-0.1710502754125871 3:-0.309850483152607 4:0.255642456909988 5:0.9949117714165621 6:0.12553772251510864 7:0.6412602805648968 8:-0.6225679446416825 9:-0.15867011477056936 10:-0.4970695349036196 +-6.931030745953174 1:0.5151452174260762 2:0.10077064818539072 3:0.9515221270405545 4:-0.21551878535257907 5:0.29152528087481366 6:-0.10995497026133605 7:-0.7872786530801681 8:0.9909149980139627 9:-0.6044617953251021 10:0.4135285912642448 +15.538062451207367 1:-0.493569696351595 2:0.7280914440594639 3:-0.5399160539735497 4:0.5688018985826291 5:0.8296550361854862 6:-0.3519274619833537 7:-0.5536583684230114 8:-0.9648774930921231 9:-0.2649670832738824 10:-0.2337289004188019 +9.499729032920945 1:0.22017490770298553 2:0.7693082799289328 3:-0.7645745307823122 4:-0.4243400515554365 5:-0.7065281515163817 6:-0.9488470141298047 7:-0.7888781431404843 8:-0.38027758953310964 9:0.11329243985448345 10:-0.5636550498916204 +-0.6039115764951412 1:0.3128791250125589 2:0.4690308315665288 3:-0.9819748103687955 4:0.28931283693913223 5:-0.6283983933456656 6:-0.10795935596621975 7:0.7785831799196448 8:0.4453768248295542 9:0.4055410615499917 10:-0.581108383985806 +9.682301463907875 1:0.5039970331368235 2:-0.008965105921562966 3:-0.5415225380115054 4:0.4677111860370293 5:-0.3854089758945243 6:-0.8468317339287676 7:-0.29258253017713587 8:0.7361173598968789 9:0.5722561668394952 10:0.8524030171340933 +-2.8752191903928064 1:-0.45407356732165205 2:0.6563221064539377 3:-0.8938366926767671 4:0.6028173420234533 5:0.6792881349943096 6:-0.6295604812779405 7:-0.21641416912497213 8:-0.8703620515028858 9:-0.3397362922228042 10:-0.0966947467107604 +-28.046018037776633 1:0.9493308195854675 2:0.3285214661535252 3:0.749300278016316 4:-0.006681618268088219 5:0.2936055273341429 6:0.0044706790416966236 7:0.5006172205470896 8:0.38751814960349473 9:0.6069735922707928 10:-0.794612882855285 +2.8752582614589373 1:-0.9443232811926943 2:0.3153126492983107 3:0.6423843271417344 4:-0.09528333043829118 5:-0.2318773828230698 6:0.32597909562645766 7:0.42808555740416065 8:0.2895959316734451 9:-0.5106491076955746 10:-0.2917418155655722 +-2.203945173593806 1:-0.13844025039418084 2:-0.024638102806725293 3:0.6114514176076162 4:-0.6939316676972749 5:-0.036549673716341324 6:0.0942395290460385 7:0.7943411369475493 8:0.7025693796408046 9:-0.21822635487138853 10:-0.6045250179827362 +-5.070655299509993 1:-0.8035156105848074 2:-0.5344928236067734 3:0.6105404604447127 4:-0.7538635525543969 5:0.9836765037886612 6:-0.5700253195942724 7:0.9232380985458313 8:-0.26374377078100464 9:0.9079431466301682 10:0.8404281771949533 +-2.540181413836895 1:0.220453181647285 2:-0.03105792440486077 3:-0.17131282366411926 4:-0.41800060634660485 5:-0.1477564564540963 6:0.055537469469941536 7:-0.8092076926316594 8:-0.29815112444525727 9:-0.20030580647762464 10:0.337865838755971 +19.341342586351033 1:-0.32052868280788616 2:0.954507993011956 3:0.38642226954792824 4:0.9240442034609888 5:-0.46077559741256824 6:-0.7991393493824104 7:0.9396232321156679 8:-0.2486930151964184 9:-0.6256485833035617 10:0.14861843824730103 +0.31398559122529757 1:-0.4684215762946897 2:0.07873308388585198 3:-0.3589594894052015 4:0.14284662079329458 5:-0.8936272055527841 6:0.5647217242826741 7:0.49613233215723507 8:-0.501698787526992 9:-0.46710107378968724 10:0.898517179577361 +12.243117462926584 1:-0.8147610562690222 2:0.21104006948075482 3:0.42405323019132957 4:-0.667965573810795 5:-0.267026607469405 6:0.7949752815579358 7:-0.07664414977654532 8:-0.6023087644686556 9:-0.659375887511856 10:0.459735946423397 +-4.623091296763939 1:0.08405646515942733 2:-0.40009448092691446 3:-0.39831245310544094 4:0.8794137836499942 5:-0.04788565812369017 6:-0.06763019434549333 7:0.41324877265674065 8:0.39746868847324146 9:-0.986729367280818 10:0.7001677710291752 +-5.782162271139417 1:0.29127970805530157 2:0.6712715787317827 3:0.27575757044478477 4:0.31525054647682804 5:0.6905016168465983 6:-0.5195319089267731 7:-0.06598129860341295 8:-0.5142554034519407 9:-0.11546331150946942 10:-0.2934524891698944 +-9.892155927826222 1:-0.7048583334456604 2:-0.46598491327111247 3:-0.02034722477413209 4:-0.663294196316965 5:0.4485329128582778 6:0.4553619594861118 7:0.7154814909138205 8:0.7532937661147989 9:0.020693077287389894 10:-0.23131986644633207 +0.5422764698408844 1:-0.1513298744027669 2:-0.4591544337339648 3:-0.7192219559850723 4:0.21236658135317632 5:0.12050445497328166 6:-0.42411528242712127 7:-0.15103925528861595 8:0.9032115729799512 9:-0.9228817525021624 10:0.2604090001033641 +4.187800872274017 1:0.3084355607627949 2:0.7029638272178733 3:0.34098344122299573 4:-0.5818421369891376 5:0.1332356708082485 6:0.22671316744441716 7:-0.6750469752494854 8:-0.4065302428716193 9:-0.48213803977370073 10:0.17918596677210186 +4.487701812297124 1:0.8352061350259052 2:0.2757393215770836 3:0.9310504392364667 4:0.519503546762708 5:0.5270245209143005 6:-0.08181154800488488 7:0.5148324302455536 8:-0.6680946101511949 9:0.7574060703813035 10:-0.4721334895419935 +-5.150140984417682 1:0.8113709439821006 2:0.21939305063309278 3:0.02109986546311826 4:0.07450107676582762 5:0.723883853128624 6:0.5392035186380486 7:-0.1382740221237464 8:0.9990201540159807 9:0.10429329766137108 10:-0.1365266408862309 +-6.544633229269576 1:-0.08278037549320039 2:0.6982730989138761 3:0.9090685953368327 4:0.6754092061339365 5:0.5889199822482736 6:0.020678619551471433 7:0.47605785660672084 8:-0.49926771127869873 9:-0.28380077002944093 10:0.5282319276258469 +7.216836352055753 1:-0.8510680074642156 2:0.42611818262128476 3:0.593607821624947 4:0.5635067468583634 5:0.2121930523769171 6:0.2708063180622071 7:-0.31491113345871735 8:0.005990053407278095 9:0.8985259402559085 10:-0.44549339042232794 +20.874246167942125 1:-0.53010692413621 2:-0.9897084749945524 3:-0.9083978261828305 4:-0.15581655583739495 5:0.9974035542095165 6:0.9894717992956665 7:-0.7287287537245402 8:0.06425127137526943 9:-0.06684164745938337 10:-0.3600621883071937 +-6.556192430758147 1:-0.7655958349167471 2:-0.08083170734199419 3:-0.8540636958251198 4:-0.09994429443696973 5:0.1734809016500265 6:-0.29563180244063325 7:0.2158497607364409 8:-0.6071644305523003 9:0.8063426715403785 10:0.47092299197899345 +7.252748885335252 1:-0.36403312429467216 2:0.1237451136826817 3:-0.5756427605741237 4:0.7612833636750866 5:0.9350628314096134 6:-0.012087843264624754 7:-0.03742573515965031 8:-0.05210460803183037 9:-0.5333214800203341 10:-0.013320030179712505 +-9.2679651250406 1:-0.5057250557539077 2:-0.41655319851679495 3:0.1897431234740683 4:-0.038318717640150046 5:0.9136495575471062 6:-0.8890525036858237 7:0.40859501498633377 8:-0.8746985847539293 9:-0.005836984002720369 10:0.7838036026237987 +-15.732088272239245 1:-0.8546867577633044 2:-0.3003980324850013 3:0.49649883896876834 4:0.710496747220617 5:0.5848510480601048 6:0.5714826756665468 7:0.5487975165953451 8:0.5654333402837335 9:0.863539315599626 10:-0.9699410102494574 +-0.20412431312519014 1:0.13323548063028934 2:-0.3030177580658542 3:-0.6358920925969869 4:0.3729380701923921 5:-0.8131818118430312 6:0.11567152703716288 7:-0.3645508535812394 8:-0.5487213252460876 9:0.5605886387366952 10:-0.8400308993051686 +10.445759684895373 1:-0.92707280355555 2:-0.9307772570299944 3:-0.11971873660640964 4:0.5140245291069254 5:0.5751145648836897 6:-0.43850910073502347 7:-0.7872208869913662 8:-0.3087975452145404 9:-0.4645849758749403 10:-0.1563641826381328 +3.349708377102383 1:-0.6334394121009499 2:-0.9008086683014112 3:-0.2678892493467009 4:0.7626514243443427 5:0.6406493676995701 6:0.3669245573649391 7:-0.052050629941784665 8:0.6713394117904852 9:-0.11458974566378233 10:-0.25949626043219576 +-23.487440120936512 1:-0.5195354431261132 2:0.8080357948412571 3:0.8498613208566037 4:0.044766977500795946 5:-0.9031972948753286 6:0.284006053218262 7:0.9640004956647206 8:-0.04090127960289358 9:0.44190479952918427 10:-0.7359820144913463 +-11.827072996392571 1:0.9409739656166973 2:0.17053032210347996 3:-0.5735271206214345 4:0.2713064952443933 5:-0.11725988807909005 6:0.34413389399753047 7:-0.2987734110474076 8:-0.5436538528015331 9:-0.06578668798680076 10:0.7901644743575837 +-3.650649176738987 1:0.9665344025238449 2:0.1395514751689353 3:0.954697162791015 4:0.2093601878355722 5:-0.42841737775246336 6:-0.02877209657213764 7:-0.8382526163632971 8:-0.03773878779258388 9:-0.3751775119106411 10:0.6477987464528951 +0.21915863046310957 1:0.25143109618049353 2:-0.06463696557011112 3:-0.3324862332340037 4:-0.7184623449423757 5:-0.8897217937178385 6:-0.7336278194091297 7:0.8547631637534296 8:-0.7582613025929346 9:0.9080481791309838 10:0.9427850135311773 +4.813247597584681 1:-0.4564689661727537 2:-0.4315414033069003 3:0.09676404446694242 4:0.6024645727173434 5:0.20466090997530606 6:-0.09432916868838737 7:0.6402934161890248 8:0.741842551426011 9:-0.343937669190693 10:0.308871619426873 +-3.0700825038127206 1:0.660084046469162 2:-0.02317305725931229 3:0.7567569356692221 4:0.2528834502236612 5:-0.3935091635208898 6:-0.9965507922509653 7:0.9065754202428946 8:0.6988037588300844 9:0.05145737657924321 10:0.4697377584426863 +9.762542323725354 1:-0.036129448543738896 2:-0.8252508992030534 3:-0.752854859129851 4:-0.9027424488033049 5:-0.4763092428375775 6:0.4832492121777574 7:-0.2935697977919014 8:-0.9197908986231211 9:0.8914359296658816 10:0.8688484670974876 +6.690913813146277 1:-0.7649833946109403 2:0.0419327356721928 3:0.5420954694310764 4:-0.7373259510045522 5:-0.9187577877864708 6:0.6431180783847401 7:-0.6272529754533058 8:-0.43356486537110106 9:0.16848266440424364 10:0.3129700315745716 +21.325049167466855 1:-0.36392795201361383 2:0.846518905511275 3:-0.26361421923150097 4:0.5140384860444887 5:-0.9147771624497878 6:-0.22044646197773576 7:0.14099760779666948 8:-0.546631395802236 9:-0.4345465263406878 10:-0.2759894364167672 +0.41237529640734055 1:0.05016964684797287 2:0.21708512805176072 3:-0.9444942733586354 4:-0.6118772896807114 5:-0.18053631846913665 6:-0.06752556529755416 7:-0.0081819952134361 8:-0.7774039956687315 9:-0.5548994336153177 10:0.7510833121912588 +-15.056482974542433 1:0.6012054064354875 2:-0.6127014811673221 3:-0.8356741843949218 4:0.19830469052767397 5:-0.07726493085289698 6:-0.5756891943805014 7:-0.49010583357941884 8:0.7493759119974515 9:-0.7828994218436376 10:0.6154265137741459 +-2.109441044710089 1:-0.5757976103755722 2:0.3686657403505862 3:0.5418762444017706 4:-0.5896052565388463 5:-0.1000712585735879 6:-0.8114188394866342 7:-0.5863884932327266 8:0.28289838755838015 9:0.5378646921099333 10:0.5063780890366179 +-5.249715067336168 1:0.6828022788286754 2:0.9044668986252975 3:-0.6010464361571437 4:0.8416122052398811 5:-0.9846446498408039 6:-0.3777762313579811 7:0.5763775880953983 8:-0.07608009385213488 9:-0.29576023599575474 10:0.8845728751981716 +6.907770824878343 1:-0.9751352215365647 2:-0.8297271715190588 3:-0.07240311280415779 4:0.4796310183582191 5:0.358213469979769 6:0.4628020211207058 7:-0.9753405605972942 8:-0.765583403709019 9:0.5623611232648877 10:-0.14725965272406616 +-9.299021854126096 1:0.8784076266914045 2:-0.8314918563417382 3:0.8701529449600536 4:-0.8070129727442199 5:0.07396877198841345 6:0.0040889707225901795 7:0.40529205456687145 8:0.6412485325027342 9:0.1443450351498905 10:0.404997568726581 +10.95643670126225 1:-0.37321642594676097 2:0.47766490569544473 3:0.9201313123144423 4:-0.649393433578801 5:-0.9084894063674787 6:-0.2547160991750408 7:0.7674649994523459 8:0.646056370118979 9:0.6014100713287893 10:-0.15130291862509182 +-2.6397202393123336 1:0.3285252466844373 2:-0.2714281159811125 3:-0.5869561846815805 4:-0.5643935541712441 5:-0.7285201267315389 6:0.6502951074428092 7:0.8611880383193904 8:0.6380425291162128 9:0.5118538704085516 10:0.4012684110865874 +12.521131042032012 1:0.4843931319727355 2:0.06440853455169626 3:-0.6151259240105509 4:-0.4180928328467284 5:-0.4607061773323424 6:0.8104775289268906 7:0.3284199695768064 8:0.8425028998495565 9:-0.34822319854822825 10:0.1969239149176112 +-16.151349351277112 1:0.7787909191620395 2:-0.5219981442072688 3:-0.7230569918898555 4:-0.05707801168212101 5:-0.8134225824740247 6:0.09950684183685454 7:0.6261274830059296 8:-0.9502006765164366 9:-0.6724983095526844 10:-0.600347212281825 +-5.039628433467326 1:0.7680701397575322 2:0.7956844224408437 3:0.4131717201035916 4:-0.3127895385265915 5:0.7226571953995224 6:-0.06845863083031967 7:-0.1007291660029832 8:-0.9130249132342207 9:-0.4605180615474036 10:0.42093879298156 +9.007596502870785 1:-0.6562175566238462 2:0.08420074013217049 3:0.589801949672486 4:-0.11964901133703987 5:-0.8145711913860048 6:0.43854302140351065 7:0.5992967124729605 8:0.253745043289755 9:-0.10742030998120033 10:-0.5993228348160153 +-12.41094640284016 1:0.31035917086763765 2:-0.8053417167237813 3:0.5754655536186164 4:-0.3645388095106201 5:-0.9135176753316416 6:-0.8690739610562535 7:-0.14039224825138197 8:-0.7112835675593987 9:0.25762942117230825 10:-0.9483300117501923 +-12.130353212287929 1:-0.41404309625298485 2:-0.7695984204591535 3:-0.44569447239245275 4:-0.3168863099965644 5:-0.26669244730409036 6:-0.33484042698895755 7:-0.41062396946367685 8:-0.09075804785640385 9:0.8511367190902208 10:0.021918606255194595 +-15.375857723312297 1:-0.9794952880997945 2:-0.9547237660069134 3:0.2460912345929791 4:0.3174335823329406 5:-0.23758562926743054 6:-0.113610303129287 7:0.18292675847568063 8:-0.9656446754474337 9:-0.58300134324846 10:-0.6689602908128025 +-6.397510534969392 1:0.440780662587545 2:-0.03737991637410243 3:0.9506435891605849 4:0.8177486462589998 5:-0.2917628929963241 6:0.42365289098031034 7:-0.4280555544979745 8:-0.18388426864865903 9:0.5057230088452542 10:-0.1699163749308643 +-9.789294452221961 1:-0.25066699970459694 2:0.1772977344415987 3:0.5913498268900952 4:0.6293756431864681 5:-0.6430441015863757 6:-0.7238519180293621 7:0.13639541626580498 8:-0.6620281401715837 9:-0.9515237061912034 10:-0.4333426289849791 +-13.15333560636553 1:0.3491978525665129 2:-0.4067353159374012 3:-0.8677040612253524 4:-0.5757086910974862 5:-0.3186886816681207 6:-0.06609938943414573 7:-0.5419747642754873 8:0.9632759660044383 9:0.2673520823110991 10:0.36463236596724546 +2.2307697392937795 1:0.12285527276472785 2:0.8938323722714365 3:-0.16995870341610209 4:-0.3298643049714254 5:0.16781582791954253 6:0.42381594687105895 7:0.9245288214717629 8:-0.08709025093361689 9:-0.14227085487682722 10:-0.2888302862659746 +5.892885365508635 1:0.10116053019915738 2:-0.41641547074900154 3:-0.3750004290914961 4:-0.5619470211369917 5:0.33343039544460384 6:0.46766042657994733 7:-0.6937940929321615 8:0.7044604392055189 9:0.8879353764416567 10:-0.5490902425042639 +-16.692207021311106 1:0.9117919458569854 2:0.628599902089868 3:-0.29426892743208954 4:-0.7936280881977256 5:0.8429787263741186 6:0.7932494418330283 7:0.31956207523432667 8:0.9890773145202636 9:-0.7936494627564858 10:0.9917688731048739 +10.454641756541454 1:0.3490213088098768 2:0.6103387992494194 3:0.6753935651135747 4:-0.39560763769937934 5:-0.3042308221531884 6:-0.9283481899557042 7:-0.7904038212853011 8:0.37488335848537346 9:-0.296477977723397 10:0.30894819444660304 +0.08978797103855778 1:-0.13445409764877803 2:-0.6404150831493631 3:-0.24740260669490133 4:0.031151119464385646 5:0.9207882173498612 6:-0.6146471129497393 7:-0.9736175690408087 8:-0.2673180325645341 9:0.5800384183301572 10:0.479811220263183 +1.7362099941626894 1:0.5171681395917551 2:0.6177735922313075 3:0.6446678302226738 4:-0.5731769722311459 5:-0.2686270617709168 6:-0.6048534221658814 7:0.7002124303669326 8:-0.1479765297345712 9:0.009254061109394307 10:-0.31519081920853287 +-1.0349488340235453 1:0.612980711993536 2:0.05771318707554962 3:-0.10821368362160744 4:-0.8755559420458141 5:0.42566546089913326 6:-0.7966341558699277 7:-0.45253617234374466 8:-0.8289517557653971 9:-0.8968075137250837 10:-0.6325457096866376 +0.10157453780074743 1:0.9143592240573388 2:0.06419631741815457 3:-0.9961326744227916 4:-0.47174548800139715 5:-0.0821464027819967 6:-0.5495006555498168 7:-0.5627911401420294 8:-0.43426056724099005 9:0.892026786364895 10:-0.23546485121284055 +-12.92222310337042 1:0.218687524173371 2:0.013626751799176162 3:-0.8372219908323961 4:0.6197296846266354 5:0.7429130827811232 6:0.48009972886541896 7:-0.35667717521227904 8:0.18337067878780533 9:-0.22935396092245197 10:0.4076715024284059 +22.923352376063196 1:-0.7522075505725567 2:-0.20686029838909326 3:-0.1386664769095396 4:0.157117595808127 5:0.9462377653889174 6:0.9182504509330662 7:0.18170057747293833 8:0.27735387813088863 9:-0.6355799944714868 10:0.9764849106195284 +-6.132450015997121 1:0.2822534275343054 2:0.2625905791399692 3:-0.02565260641304068 4:0.4891221076432757 5:-0.6426178913585772 6:-0.8999539149461033 7:0.12659507663825287 8:0.5889572439755832 9:0.49107548332672857 10:0.47595749470452 +-9.173693798406978 1:0.4430245286298278 2:0.9923116639471541 3:-0.5609082824097824 4:-0.36542266258313916 5:-0.5814039716882617 6:0.20413852042674874 7:0.6097541611931963 8:0.5743002479324253 9:0.4735459963431561 10:-0.053969823043886755 +-5.814408490931223 1:-0.9206287328000513 2:-0.48298486023273157 3:-0.8477202041890262 4:0.5801385102362351 5:0.7146074564553095 6:-0.5987672678579339 7:0.6829077928212723 8:-0.3522788540815065 9:0.7729595638821951 10:0.5264904880591215 +6.474329501040298 1:0.6914309300550991 2:-0.4507700505202725 3:0.713821440501512 4:0.41599059910235847 5:0.507160951750409 6:0.8635615811574222 7:-0.6235518270244333 8:-0.5336201820384283 9:-0.7989630679361768 10:0.837293162455248 +6.984517471584806 1:0.16745919469723392 2:0.018033079961716103 3:-0.7339201095541323 4:0.17042828693740697 5:0.4493471632580528 6:-0.8938445962323078 7:-0.3206968104792325 8:-0.616617071238893 9:0.9327878222034172 10:-0.6575294247048245 +-12.479280211451497 1:0.9769767754725367 2:0.7706430863248943 3:-0.4506244622476816 4:0.12921761745628713 5:-0.0697804449658812 6:-0.7702703569987461 7:0.017734558413919688 8:0.7216294158911261 9:0.42547357862241886 10:-0.9001915116155741 +2.8363866587728186 1:0.11478724114928918 2:-0.4679790550082039 3:0.2344912687736711 4:0.5524878060045462 5:0.5252859884051309 6:0.5080674087215156 7:0.5010449021825665 8:0.048046765816400105 9:0.06654581719548891 10:-0.5801934713347348 +4.186809777233374 1:-0.02335342201396018 2:0.9035437912091193 3:-0.9283585631882163 4:0.454351316397237 5:-0.6948564428085262 6:0.11495485234890368 7:-0.23683956078769963 8:0.6442534752881419 9:-0.013866407845647188 10:0.23369602940650736 +2.8235031660626415 1:0.5609344938188046 2:0.3449103464885612 3:0.03972169049525687 4:0.31858762565827137 5:0.4409953589124853 6:0.22836189275697016 7:-0.1497811991899889 8:-0.23248048920679265 9:-0.30066618281100177 10:-0.9247232456911632 +6.96223432848425 1:-0.8160398553437558 2:-0.8212180893749699 3:0.7728655115832999 4:0.02387973088796369 5:-0.043499804905828166 6:-0.6997726250046865 7:-0.8686633773265577 8:-0.12597318402253976 9:0.967018116368416 10:0.5951339624149812 +4.669684795838683 1:-0.32226903644852833 2:0.5465858078942492 3:0.5228467793266189 4:-0.013157722224545143 5:0.5810668818928995 6:-0.1372653090293532 7:0.6446157527288279 8:-0.06005754873230629 9:0.014302180040152379 10:0.43474245441042636 +16.112744845653285 1:0.37257742858083365 2:0.19398954512844124 3:-0.11860882189887478 4:0.6492510749703395 5:-0.41273736981203313 6:0.18643017041815835 7:0.29136917186214384 8:0.47602883023389 9:0.7126916980867937 10:0.48462508659691483 +-9.196003366226202 1:-0.7263358951920722 2:-0.8503799288093836 3:-0.3120563620589105 4:0.3925562655164563 5:0.027666662972283484 6:-0.35173134138805406 7:-0.32703527910354757 8:0.3060102722285065 9:0.8609161725740202 10:0.33394557004432923 +1.242972458167591 1:-0.9029238804456814 2:-0.6392681059531908 3:0.8940879647942577 4:-0.8807357173896475 5:-0.13628130467470512 6:-0.5487534785116224 7:-0.40270307148061346 8:0.09152108686997096 9:-0.20745066734844642 10:-0.20624830574384978 +3.453659210660726 1:0.2710596844435682 2:0.6510497900145247 3:-0.2899158136103117 4:-0.13531811694554707 5:0.6965847786422426 6:0.9105343028780231 7:-0.007340232468413754 8:0.7672537187738411 9:0.3538906829188173 10:0.35387524540947646 +-0.48115211266405217 1:-0.17943755364759517 2:-0.1384979591151625 3:0.8425773648797268 4:-0.43234064993405097 5:0.919754442523921 6:0.8390197802990036 7:0.43890653121452683 8:-0.7647648217789051 9:0.14770258954363835 10:-0.6681813635676657 +6.965069440749298 1:-0.9158261471030473 2:0.5228494114644282 3:-0.07760531122743153 4:0.6154296244963067 5:0.5231830145381096 6:0.4892535590799165 7:0.1987053183082137 8:0.9995670294711712 9:-0.2020375688074112 10:-0.7853579334836087 +-1.6896486293598596 1:0.4638529147853421 2:0.0953805943546191 3:0.8506904243225251 4:-0.028262644692445438 5:-0.9462342015500664 6:-0.6934738957112123 7:0.601125018257533 8:-0.04871041957758315 9:-0.015245062056267411 10:0.6119856200040805 +-1.763729644326212 1:0.5376618752928528 2:0.8062119856717131 3:0.44996834959923593 4:0.9917728248530817 5:0.5974717482179492 6:-0.406972851600659 7:-0.8523198502065281 8:-0.3076377139692321 9:0.9099974915864462 10:-0.43374966692373484 +9.012829566937228 1:0.6885456531832366 2:-0.0631164354373237 3:0.8394182300770314 4:0.7207913383891218 5:0.4715324450375691 6:-0.34417503908167757 7:-0.31448279255342126 8:-0.020591617987411936 9:-0.37668573574418107 10:-0.6528048324896532 +-15.951512565794573 1:-0.6112828771933607 2:0.4867007149846869 3:0.863494046941478 4:-0.7292072742454481 5:0.6338749652624007 6:0.5980798993978542 7:-0.5119002889878654 8:0.8394383182101366 9:-0.1412423080445726 10:-0.15838730884968655 +-0.29622788243318465 1:-0.9436253326661384 2:0.2907259958032098 3:-0.1530538226933904 4:-0.6174176535420375 5:0.8209632215649141 6:0.5060548803172731 7:0.8212448453211292 8:0.33506684706740386 9:-0.5408309869188785 10:-0.8105966349150977 +-7.683213587039055 1:0.2525015766703558 2:0.6417869320191234 3:-0.7569571597336913 4:0.5265130776924394 5:-0.03992944660560949 6:0.18292946303778823 7:0.4286344960738724 8:0.9158523573288766 9:0.5039796366711773 10:0.27660486075533797 +3.9061298856792797 1:-0.6501789225392032 2:-0.6040685518173872 3:-0.6448094322678659 4:-0.2019498832769746 5:-0.5302977370883424 6:-0.010754341856880067 7:0.8791702222974846 8:-0.2283571791337704 9:0.4726320486679656 10:0.3413255179758332 +12.928385148211825 1:0.7793178379505685 2:-0.5207562047491976 3:0.37253320760898934 4:0.7540757518052998 5:-0.679378421540417 6:-0.11966022036636881 7:-0.4317798870297489 8:-0.004211291952602059 9:0.39024653887361693 10:0.45391057946097146 +5.787566514603203 1:-0.20596730554338039 2:-0.8840796727164746 3:-0.749416279057892 4:-0.5511023306046077 5:0.9941631901218697 6:-0.09907966722992234 7:0.701617914811792 8:0.9696055014561289 9:-0.7083648075748707 10:0.5781111533720358 +5.701262468657861 1:-0.7066995012593675 2:-0.6756815056791965 3:-0.5720277255842998 4:-0.09218662060241067 5:0.21494136076896653 6:-0.37012884573008153 7:-0.6828277646796448 8:-0.10038134655965236 9:-0.46253754509583356 10:-0.20813933595648115 +0.9473494330088033 1:0.6876806675510589 2:-0.9530860102792402 3:-0.4043172626863887 4:0.6696455505098386 5:0.17863581804857254 6:0.1944646561635497 7:-0.5283662172535679 8:0.4872263841818012 9:-0.2882651789318431 10:-0.06293411605141874 +-2.6834375589185675 1:-0.22376759986120187 2:0.36555755546798885 3:-0.5223502955721961 4:-0.20702347869224624 5:-0.7745351063999764 6:0.22879328233099971 7:-0.5440007473902635 8:-0.6959483071829207 9:-0.131433881760733 10:0.2764225554693165 +-3.2766108642276146 1:0.0304613976530983 2:-0.3148062986719251 3:0.24950420590071953 4:0.7152023826801459 5:0.9656885739650887 6:-0.3210562623763835 7:-0.7305896664502614 8:-0.49074917893875836 9:0.7802670253347352 10:0.8667409958355992 +-1.1838791995691869 1:0.06642047806096318 2:0.5336148776806793 3:-0.6199614859883396 4:-0.15342280723497237 5:0.8407250402808968 6:0.7060811811107444 7:-0.2913182140909305 8:-0.5925203360011633 9:0.22644925021629692 10:0.42395071889002467 +-1.5856680515554806 1:-0.8724712788102853 2:0.11445744032031424 3:0.5483166457680566 4:0.9469521544884028 5:0.2541682828467746 6:-0.436750733871873 7:-0.9001249399695319 8:-0.7555793441458385 9:0.06946992897983018 10:0.9724148045760346 +-13.039928064104615 1:-0.558607026518148 2:-0.7356765018678253 3:-0.7547644426290201 4:-0.24898664843938745 5:-0.3606374046883567 6:0.5836652368902306 7:0.8497678666873467 8:0.21331875915717635 9:0.3558733809635668 10:0.9642603628738968 +-17.428674570939506 1:0.8562209225926345 2:0.7077202100653552 3:0.7449487615498371 4:0.4648122665228682 5:0.20867633509077188 6:0.08516406450475422 7:0.22426604902631664 8:-0.5503074163123833 9:-0.40653248591627533 10:-0.34680731694527833 +13.886853032969585 1:-0.6354915752033683 2:-0.9132338112681755 3:-0.4816479770266455 4:0.5448417181244594 5:-0.6250746297187781 6:0.7410618768880199 7:-0.18029029550083675 8:0.777358236920447 9:0.9625064189449102 10:0.048040935468046 +15.61684729251139 1:0.2980237970192188 2:-0.8160931971814265 3:-0.29649852157138445 4:0.3896688599904572 5:-0.17552110506337826 6:0.8721328328445139 7:0.48984799668438916 8:0.9984496052876473 9:0.9665885195526289 10:0.8966559812150274 +10.33625540376971 1:0.09939495068155724 2:0.9790332181038015 3:0.9483428886275702 4:-0.5717299810793317 5:0.4876405069057712 6:0.163962913892302 7:-0.4095537988924203 8:0.8608269751255508 9:0.010028680058212114 10:0.9095786494455713 +9.706032970113723 1:0.7687898546315146 2:-0.9825109379412285 3:-0.5423211794439926 4:-0.3099509487314134 5:-0.11561305536236333 6:0.9012327035409926 7:0.5257495475790148 8:-0.33804422025989433 9:-0.144428735681567 10:0.28019332199039604 +6.189043888072968 1:0.13246655756059478 2:-0.751192382628302 3:0.2233421456265161 4:-0.1933575076984373 5:0.8681727702736863 6:-0.7656847407654899 7:0.1033145549916572 8:0.33909210370257403 9:-0.22241363302770267 10:-0.14479004187830435 +-8.680225911784335 1:-0.07718769939880432 2:0.6702228057326558 3:0.6647810334933819 4:-0.05115658747070784 5:-0.850780588302118 6:-0.040961453376221924 7:-0.8407690297644956 8:0.33775829053563156 9:-0.45421556034898547 10:0.8238500771967823 +-9.42898793151394 1:0.8925906426831107 2:-0.6771269725125597 3:-0.11635105688280678 4:-0.7266044201050157 5:-0.6902918845825077 6:-0.5911234800910024 7:0.49395074569300657 8:0.43660804414878274 9:0.8736983081269782 10:-0.8001177058312081 +8.486245765579415 1:0.5614295382716652 2:0.3972427851719582 3:-0.276268504977494 4:0.7803448249454739 5:-0.358957923558495 6:0.3477822689529795 7:-0.7944805581842691 8:0.8356932134547437 9:-0.4783293647580624 10:-0.2522633417723845 +-1.8722161156986976 1:0.11831037290857482 2:-0.7309091607574014 3:-0.7339122716951587 4:0.2046641765436359 5:-0.9914679283125301 6:0.13518339528098555 7:-0.9760821540963867 8:-0.6080636193563043 9:0.3890502262427238 10:0.33864957953815145 +0.5122357093733743 1:-0.9555852441641726 2:0.4754771858792488 3:0.3743376249200432 4:-0.2651772997462427 5:-0.7915484529586028 6:-0.7575915279708862 7:-0.10432268807273859 8:0.021604934223709238 9:-0.6458011732912265 10:0.40773716196391674 +-18.845922472898582 1:-0.6031480148285926 2:-0.8736524730197766 3:-0.311456616524979 4:0.420921703897325 5:-0.2904011177124777 6:0.6683252350591937 7:-0.3436202976676894 8:0.5023604359385605 9:-0.33056149241985633 10:0.5168854058825227 +6.492106438811399 1:0.7824832256885428 2:0.6105456307389117 3:-0.0436873997963223 4:0.46730493583332855 5:0.2057529813440686 6:0.5738310686722767 7:0.6307964411259019 8:0.6208424783086652 9:0.8931894299284251 10:0.7164648197763028 +-1.6472226859532182 1:0.8854767145642171 2:-0.8175744681485637 3:-0.14894858038610903 4:0.9667400540136402 5:-0.3575837217508149 6:-0.9211342680517054 7:-0.956785876301889 8:0.6558217028031554 9:0.8014538160668165 10:-0.9475520920917395 +0.185861229793925 1:-0.8181719548530746 2:0.9990094335332504 3:-0.8195848911987829 4:0.6991933015233858 5:0.07295718417836583 6:0.5968996100546737 7:0.4871410306452193 8:0.2980483098540927 9:0.779953293728507 10:-0.7978867112395516 +-5.973450525185694 1:-0.975435413991927 2:-0.7832951303253313 3:0.5098999023442101 4:0.46795978867990007 5:0.2538986807863044 6:-0.8182887550010198 7:0.8335391734637112 8:0.4286082996234335 9:-0.1726765956719154 10:0.7649845978453362 +-12.773226999251197 1:-0.383327656965585 2:-0.9439560491389036 3:0.25039001869622446 4:-0.9342091044843222 5:0.8711023711291135 6:-0.6027135241543655 7:0.9456874780319795 8:-0.243290468946338 9:0.625765915285031 10:0.5160550067618355 +24.290551295953957 1:-0.8368553572749229 2:-0.5859456648150321 3:0.873779532007048 4:0.7462623178738954 5:-0.08133011570245352 6:0.36767541461776676 7:-0.33129619282275047 8:0.6104289727615573 9:0.9416581563055089 10:0.18201841676606856 +14.490247980976621 1:-0.4765937762114507 2:0.16430711839945555 3:-0.526776940706293 4:-0.6802269991653915 5:0.40748236413299344 6:-0.500290139207977 7:-0.31915972151663885 8:-0.4586068416002418 9:-0.15572660263944127 10:-0.32925702602833073 +8.377230871265601 1:0.44141613060964846 2:0.1582267687752743 3:0.8760950367284166 4:0.40434058393690364 5:-0.7063758409891474 6:-0.616055773516162 7:0.996372393127579 8:0.6142084876085476 9:-0.528320587432094 10:-0.2815909691094802 +-3.2987560995836653 1:-0.4600479783378091 2:-0.04201794336103326 3:-0.8934505203905587 4:-0.44991326751905536 5:-0.5220579476363783 6:0.46060949186328703 7:0.9169289030735643 8:-0.022458426893944283 9:0.08100795210565637 10:0.5726732415540354 +0.3422568955736137 1:-0.9888686059817204 2:0.22752298580182706 3:-0.5048696915520232 4:-0.059433420464226616 5:0.7823831512651716 6:0.9865977573980389 7:0.9164100011124972 8:-0.3638554550863984 9:0.3038282907667611 10:0.4652367033461571 +-8.24116881862084 1:0.7565819250331731 2:-0.3733277500524168 3:-0.8841150081071696 4:-0.922282989989148 5:-0.041520813551309876 6:0.8615967014876558 7:0.8474207144091339 8:-0.7518437864641427 9:0.45076605239968837 10:-0.48912984167595375 +-4.367083147104942 1:-0.276459380002813 2:-0.957555271384241 3:-0.3761632810202544 4:-0.3897414804149022 5:-0.3133861519856074 6:0.0777990809172171 7:0.6638552243422928 8:-0.3477312155364247 9:0.5934885465182675 10:-0.5238903641193555 +1.9280240152322783 1:-0.40051093785549696 2:0.5070348672240661 3:0.7506759969575532 4:0.5042104954516786 5:0.9959688260926507 6:0.4657024999761399 7:0.910611131925299 8:0.9836517468598804 9:-0.6263172749113686 10:0.16955852322929155 +8.918138317441574 1:-0.22407391224687023 2:0.5545084933214972 3:0.6335932367683528 4:-0.2786481116648991 5:-0.9549992830441785 6:-0.5577873948545062 7:-0.960657200286197 8:0.3709573488946196 9:-0.9191180485753339 10:0.5033478020271929 +-5.657796797481157 1:0.6359910361030725 2:-0.1742637774815281 3:0.39699327107265137 4:-0.9841991491194473 5:-0.622093571871533 6:-0.5433497301426455 7:-0.6731178481686009 8:0.930615153085582 9:-0.3065877908950827 10:-0.5456093749639228 +8.697079562319692 1:0.4815820396629933 2:0.1173457441514223 3:0.7313645402039386 4:0.3354835387237334 5:-0.10300554535074702 6:0.5116687640761355 7:-0.8850803659104614 8:0.10654026377571157 9:-0.864976708975602 10:0.01345035085413615 +0.033954684723234596 1:0.6703241653088159 2:-0.13447915740201166 3:0.026022550037831937 4:-0.5145659862194116 5:-0.6963587636078901 6:0.652083884947352 7:0.22644722530715278 8:0.2671580129293405 9:0.9659035105360283 10:0.9547989197693989 +7.359108382166921 1:-0.6855762478384229 2:-0.7543318537260015 3:0.4772611975128618 4:-0.5588002332845741 5:-0.24271386844336496 6:-0.28595644325868896 7:0.8732728098501104 8:-0.8026384804471058 9:0.7589508830210041 10:-0.9992933613402135 +4.953597303754355 1:0.8915633023548608 2:0.04688596266450751 3:-0.26866754730613374 4:0.16694236975718102 5:0.23465297255622608 6:0.36488427850844407 7:-0.06717041145276781 8:0.9470029805221898 9:0.32483835237272674 10:-0.7892521260150298 +0.683536559775105 1:-0.32176084249781556 2:0.5446298870866526 3:0.4095848716057642 4:-0.42579711490120187 5:0.4482850543749355 6:-0.0982243826242506 7:-0.9190317048427039 8:0.06234509402976718 9:0.21327512416175054 10:-0.38023673796734525 +-28.571478869743427 1:-0.4597184465402242 2:-0.5489429386926741 3:0.33422914572951634 4:-0.15992695377395516 5:-0.7310003311728188 6:0.18241063863467488 7:-0.48385214010599453 8:0.08139879039334552 9:-0.8401239538877046 10:-0.8896372220209929 +-19.884560774273424 1:0.4619217451285318 2:0.28157115824800005 3:-0.3829811521605375 4:0.5802544015450464 5:0.1117061271473403 6:-0.8926034502584623 7:-0.34862293810401956 8:0.2733254857260612 9:0.6514176550598809 10:-0.02758604919357066 +-17.494200356883344 1:-0.4218585945316018 2:0.15566399304488754 3:-0.164665303422032 4:-0.8579743106885072 5:0.5651453461779163 6:-0.6582935645654426 7:-0.40838717556437576 8:-0.19258926475033356 9:0.9864284520934183 10:0.7156150246487265 +-15.86200932757056 1:-0.6341453831788726 2:-0.9259180639727085 3:0.302702923864538 4:0.749555004323947 5:-0.7932989575334761 6:-0.5620972938631934 7:0.020542041027870717 8:0.11610338700447698 9:-0.7912600154897766 10:0.5108307672038874 +9.027804254487519 1:0.1746878011084212 2:-0.5872807344913673 3:0.6018547246457264 4:0.5106104933121229 5:0.7329523371170135 6:-0.40058771577765895 7:-0.48753463550174025 8:0.34308791976318 9:0.3407668956765344 10:0.5964472848798394 +15.949172086880687 1:-0.7790584545657173 2:-0.017224094786103317 3:-0.0974907790179953 4:-0.10287391996036166 5:0.6007953354774878 6:-0.7032497754397848 7:-0.36068070856329437 8:0.021391994204512432 9:-0.6509100388083549 10:-0.5410899936281377 +-6.151586699415245 1:-0.5318094974022525 2:-0.830796057445983 3:0.603828597318087 4:0.6660892552257192 5:-0.18529748408390523 6:-0.47166833767648986 7:0.592915541856605 8:0.9944601563352204 9:-0.6981606574244703 10:0.34942553665003584 +2.010398523297265 1:-0.9293899922307269 2:-0.07588009904844029 3:-0.8500855420709359 4:0.12191867923536615 5:-0.528778681165414 6:0.3117086447237414 7:-0.4222963938187163 8:-0.03247894950300623 9:-0.05387792412717962 10:0.4053568741659812 +-6.749023248121471 1:-0.9875370165216966 2:0.7137693455001415 3:-0.2510160963160164 4:0.8732150877079123 5:0.49658934612905314 6:-0.9817012857861731 7:-0.2045309437850289 8:0.7562713668333418 9:-0.6787434327188155 10:-0.6147932888026117 +4.452639829999693 1:-0.35256148944834176 2:0.7581152951164591 3:-0.37755890552299265 4:0.9480813371197343 5:-0.3419340388717347 6:0.3487602851799074 7:-0.5576726724270562 8:0.4899696188087421 9:0.563074979676983 10:0.7865891460062227 +-4.938733988900586 1:-0.4108386466193119 2:0.3287655432069885 3:-0.5853553038038923 4:-0.6480591422742821 5:-0.4787998161299789 6:-0.5828003484675421 7:0.42835744317623003 8:0.8378098987706633 9:-0.5645180498703375 10:0.28981512694646705 +-3.373242544176224 1:0.04989033652617936 2:0.6575826440927308 3:-0.24028051935833128 4:-0.6649808138961095 5:-0.6530198970442704 6:-0.19331254127919362 7:-0.6743004878881749 8:-0.7214986105015062 9:-0.30648035516261385 10:-0.6455097687924254 +-3.2843694575334834 1:-0.3548536057581908 2:0.7350125943559394 3:-0.3635282827378974 4:-0.8552820154885781 5:0.9140879208466111 6:0.21870365067770892 7:-0.17738543429561382 8:-0.052851966578491005 9:-0.36066059517759097 10:-0.9020765799355679 +-3.277146077677404 1:0.910961221014513 2:0.4302525202590246 3:0.11079959840001119 4:-0.3614188274820125 5:0.5080231397310961 6:0.013940825892631237 7:0.33583012240022403 8:0.5008797094229163 9:-0.663083147090173 10:-0.0865028013627418 +-0.202246147968096 1:-0.4929308143227653 2:0.8374300027105082 3:0.08763999085193186 4:-0.499738438136623 5:0.5926071511295365 6:-0.5135396038023627 7:0.6946715869746543 8:-0.5184428793490325 9:0.21753085495829239 10:-0.33796308746585235 +-7.1237150573506955 1:-0.8506203499039495 2:-0.6581804183622855 3:0.6484205342724825 4:0.013914696389758285 5:-0.6214530117645831 6:-0.011163110491807293 7:-0.6025372583334574 8:-0.0371573886520411 9:-0.7933455929226487 10:-0.38653838674273455 +6.298226129171093 1:0.7304191211928768 2:0.8128475475660479 3:-0.03161148630216015 4:-0.6018899317958344 5:0.19277055729934367 6:0.3002272616310928 7:0.949169758830406 8:-0.1011823256970481 9:0.16093341376629966 10:0.9596833606094763 +14.906594657519511 1:0.5053240355803015 2:0.6775698974866082 3:-0.6194771000646291 4:-0.02876927004033525 5:-0.5481504206112477 6:-0.9239150546263386 7:0.471216755072994 8:-0.0027794620943384363 9:-0.8954411386878227 10:0.8991742143686698 +2.1710965297686267 1:0.4578509053930304 2:0.9270194505165124 3:0.22470373699901236 4:0.21526179917432753 5:0.5299563895862103 6:-0.5824108997775908 7:0.03801922095671095 8:-0.5164033454609385 9:0.4370246809487237 10:0.6514133050988229 +15.05806598279517 1:0.48645077410559057 2:0.7821442063987365 3:0.1943681666933883 4:0.8289246958621577 5:-0.08034311437806041 6:0.03709694472527203 7:-0.895481297246602 8:-0.42921579749551664 9:0.5447075872378688 10:0.844397849728866 +-0.4683784136986876 1:-0.5083135683360327 2:0.626070365769088 3:-0.8737725909401557 4:0.725622293853621 5:0.0018794384199978253 6:-0.9343604622552886 7:0.6655593328822609 8:0.47501755618845753 9:0.8388618477210947 10:-0.5143806767304449 +5.823027255871114 1:0.08635467091841886 2:0.6314532702073175 3:0.8862069437865836 4:0.6542025864928516 5:-0.6846784290231471 6:0.048487096050569445 7:0.30828004933669395 8:-0.49438881988995687 9:0.5706936923061823 10:0.037705651885639346 +7.03779380408974 1:-0.07193682621291098 2:-0.5816975957307158 3:-0.8426927090342973 4:-0.37504851992255306 5:0.4473129018316815 6:0.3101938194888525 7:0.6160050428837607 8:-0.913998555949695 9:0.40461966540531313 10:-0.7581141330823786 +-9.770500546345563 1:-0.31358873581579894 2:0.11771478839130278 3:-0.3404842110585631 4:-0.0604362797252429 5:0.2159524972176814 6:-0.24737863017398087 7:-0.8541428610709716 8:-0.06753562283135062 9:-0.11567537916769255 10:-0.5606246203677223 +20.000154367451547 1:-0.344717847914646 2:0.8454969480099985 3:-0.58856299370874 4:0.5884510299634649 5:0.49162879631128553 6:0.7958075013181658 7:0.7781911267315837 8:-0.6780885011989877 9:0.9797694629597928 10:-0.1872163682079866 +-6.239848349456753 1:0.9132793720646253 2:0.1680340663118458 3:0.01740115925682284 4:-0.26580395408599133 5:0.28551914590761074 6:-0.9939706142381568 7:-0.8740927279520219 8:-0.8731218126652498 9:-0.10993630739903892 10:-0.3069565039708746 +-4.173072569004537 1:0.7864835254860851 2:-0.5614522227484218 3:-0.7718396381376464 4:0.49508673889127985 5:0.24030155936964714 6:0.8080778221819038 7:0.05395496402881128 8:-0.3045148076729973 9:-0.6134406357458853 10:0.7447268183581948 +-11.328415936777782 1:-0.10183127796258096 2:0.5689039487721601 3:-0.07015335898840225 4:0.23254189629731292 5:-0.3226974656715038 6:0.2859450214054784 7:-0.4916677058012495 8:-0.27564895614732055 9:-0.9416483232894219 10:-0.7472248333434015 +8.719164753818454 1:-0.8231424386390782 2:-0.03953537069863633 3:-0.3271580541537027 4:0.892192314973022 5:-0.6759017192358232 6:-0.419591686354591 7:-0.23967385135363606 8:0.936992531568956 9:-0.12946409158671512 10:-0.9082863469271643 +22.31738046492344 1:0.37030851555335365 2:-0.06654751559177563 3:-0.5759425437665169 4:0.9179952251152963 5:0.8628921839116359 6:0.8421952184405965 7:0.9625804174561126 8:-0.03075332253237728 9:0.12227386374957994 10:-0.6243390357793757 +-1.189108450798179 1:0.5681776913545951 2:0.46049028271139436 3:-0.366463711956754 4:0.025856437432560275 5:0.7547565372954261 6:0.5506193192167212 7:-0.6279807084274867 8:-0.38698884324386107 9:-0.9885778854008227 10:0.7814740172261654 +2.8767042393531965 1:-0.6841229745503388 2:0.6252203895646273 3:-0.6737644654353572 4:-0.7321040107741059 5:0.3162570540986238 6:0.6211089085315002 7:-0.33984617437403464 8:0.1227089818682312 9:0.04586594421613177 10:-0.4679977358965799 +2.783332151730615 1:-0.39148258540779013 2:-0.3037233649803406 3:0.7955133548911926 4:-0.1729544208044842 5:-0.18247049275020033 6:-0.1315085429729259 7:-4.447133918370483E-4 8:-0.805837119503338 9:0.11574866650006688 10:0.8517519041042676 +-8.99205564094827 1:-0.45501536967706535 2:-0.35829694693457914 3:0.775695048377375 4:-0.25331195582275745 5:0.15524612858817055 6:0.7400717904631442 7:0.8382485596668376 8:-0.5619009369436814 9:0.4386801597659249 10:0.09960232210246622 +-9.808386702564658 1:-0.987404834666963 2:-0.6732308850750186 3:0.5528285725528492 4:-0.8796302275267409 5:0.30705569958232193 6:0.8635312232105203 7:-0.14033675947074187 8:0.5516086773506235 9:-0.7487899106678442 10:0.8851518933134919 +4.948281656077033 1:0.4331269064492329 2:0.4628446087354616 3:0.33730748244242537 4:0.3473124014683382 5:-0.1707966473106064 6:0.8558057784524846 7:0.1390312032172829 8:-0.7918343112673001 9:-0.85993782695915 10:0.33563174747577107 +10.791261476321019 1:-0.5417345768902055 2:-0.06334901799780424 3:0.027652223245870466 4:-0.9881487640651161 5:-0.19441123027957707 6:0.40295156581142355 7:-0.8315553696517317 8:0.11405283165483926 9:0.5377980570161418 10:-0.24581620554740824 +-0.7287230169119936 1:0.33985587202063283 2:0.6841261099887705 3:-0.9441564997438197 4:0.28660913255058906 5:-0.7597915572726905 6:-0.8535957517473378 7:0.609134673753593 8:0.29636368731717977 9:0.05791523580926916 10:0.5589907965230858 +-26.805483428483072 1:0.4572552704218824 2:-0.576096954000229 3:-0.20809839485012915 4:0.9140086345619809 5:-0.5922981637492224 6:-0.8969369345510854 7:0.3741080343476908 8:-0.01854004246308416 9:0.07834089512221243 10:0.3838413057880994 +-16.71909683360509 1:-0.24375714099465773 2:-0.11915875769929496 3:-0.3741442802364221 4:-0.3812947578178094 5:-0.7032156297055756 6:-0.18339122712542388 7:-0.8634662520461855 8:-0.714561692659166 9:0.020558676493369177 10:0.22804428969949986 +-8.822357870425154 1:0.39332200105884363 2:0.5652370435795515 3:0.6220479966351453 4:-0.018976695481651484 5:-0.6868425195058918 6:0.2029750380170401 7:-0.5550873767310935 8:0.16864133648532342 9:-0.008843355054633628 10:0.6472547984399621 +0.36392761004065594 1:-0.9059630492963144 2:-0.41039282402227384 3:-0.006673269562094131 4:-0.4989314017618798 5:-0.17726034513032318 6:0.037764439388023874 7:0.30703957185016595 8:-0.09040426404909185 9:0.38661451965066274 10:0.1630571642147851 +7.415902871490132 1:0.188586850708651 2:-0.33013604761672566 3:0.6667976416858177 4:0.8537064956198137 5:0.03971370422819254 6:-0.43229195778759966 7:-0.9607154505216515 8:0.8413204878098277 9:0.40010565279599897 10:0.7306602852367441 +-4.129456164370826 1:-0.7967510984807558 2:0.545111159425699 3:0.16038228447433012 4:0.6311115528116698 5:-0.01985759480036542 6:-0.9516543115476572 7:0.18022912194075458 8:-0.2177157123823752 9:-0.5433158910016767 10:-0.4603867691069983 +-9.211066571082247 1:-0.3611235296125135 2:0.1402619601475985 3:-0.23132525512647795 4:0.5534401725834837 5:-0.34978585787763206 6:-0.24147682088922773 7:0.8089009287617064 8:-0.09075864922490862 9:-0.05759391404550773 10:0.3371306765964468 +6.52392916461972 1:0.19122050285976044 2:-0.625453376800498 3:-0.26804961781489856 4:0.9669297468261109 5:0.9142504122291741 6:0.7678963028488108 7:-0.6852943621882759 8:0.5898129788981794 9:-0.6580947533327339 10:0.46875109532259396 +-12.46765638103286 1:0.35148385951742633 2:-0.5206883134357769 3:0.35436280451876345 4:-0.8837833467474128 5:0.3433887284719144 6:0.3914771858025621 7:-0.17813796710416252 8:0.6553344538056296 9:0.3721548243590813 10:0.9442185832979726 +-4.937258492902948 1:0.9150659354384785 2:-0.17085510578573548 3:0.8233227233543232 4:0.2539669132090434 5:0.18955049451212935 6:-0.2833188558310358 7:-0.48483747414616496 8:0.8917378487725669 9:-0.13169122011498646 10:0.9815059855284158 +-0.5233425797210233 1:0.4238363705720569 2:-0.18363058784066522 3:0.2949874786744968 4:0.12235592695567354 5:-0.9746310186182559 6:-0.8990867637441311 7:-0.8580982328464586 8:-0.7930887027205957 9:0.16757307988090275 10:0.988861929608575 +-11.904986902675114 1:-0.3692990475534952 2:0.32166293883244323 3:0.3401547722249436 4:0.10009747375878408 5:0.7598877208920192 6:0.2853003389082669 7:0.22880221701675074 8:0.4521491122351502 9:0.33222018268933895 10:-0.9500018867461919 +8.324969054805921 1:-0.48086111720736513 2:0.3705524122401185 3:0.43635448766342133 4:0.6544321903349255 5:0.059000747296945155 6:0.3328036763371236 7:0.9609146376298034 8:0.5943082361322021 9:-0.3074246170581105 10:-0.6763916655761453 +0.21701641918233017 1:-0.29449708766806304 2:0.040640346437143426 3:-0.6524819533513639 4:0.37482287233702394 5:-0.29800608396043216 6:-0.537030944860492 7:0.2862394027536084 8:-0.3783043133672048 9:-0.5292179323972728 10:-0.09583783955916791 +-6.84977373580439 1:0.825136109631339 2:-0.5722868691442817 3:0.11048134523744757 4:-0.5946054293068455 5:0.28061485657354823 6:0.9135611623885838 7:0.35590421873954603 8:0.8943562249941011 9:0.4183378981109729 10:0.5714160298247304 +-11.039347808253828 1:-0.9620263418414967 2:0.22669065740934724 3:-0.7378036492234086 4:-0.4460191511609126 5:-0.2594476006347024 6:-0.989879976130936 7:0.762096015449097 8:0.6983868222083149 9:0.8729993459982626 10:0.3426647417451305 +-5.882860061103163 1:0.5247178959769465 2:-0.6217169944869176 3:-0.13640714414758315 4:0.6608201052790283 5:0.5789945243704264 6:-0.12686057623612612 7:0.7277882307863026 8:-0.47949544949858236 9:0.9781208432412936 10:-0.8980068284379361 +23.52945433069272 1:-0.12339549394875426 2:-0.6769524283089239 3:0.9324962870874394 4:0.28956947294105206 5:-0.2957355479338608 6:0.7504385350771912 7:-0.8769262306643106 8:0.41591311300668155 9:-0.7694611231426498 10:0.9885110924181837 +19.043184423383824 1:-0.13783178628851878 2:-0.853631844645959 3:-0.12792415583066052 4:0.6936898387576049 5:0.8488563282318959 6:-0.6530521292304581 7:0.27832187660440666 8:0.09838048719062442 9:-0.5913230087557231 10:0.260839433107553 +6.83105883806984 1:-0.9085282656519695 2:0.65203708247844 3:-0.687580071985604 4:-0.045008726377529173 5:0.4762107922777967 6:0.15939259525248506 7:-0.46363191848939334 8:-0.25856682230410266 9:0.313842004143269 10:0.5042938214484851 +-9.409197719620593 1:-0.34356198962701945 2:-0.06381545064099514 3:-0.9332814619122063 4:-0.2629675367181199 5:-0.03876014002851913 6:-0.4606936151803749 7:0.49272969757318563 8:0.5550196351479111 9:-0.1758425343811718 10:0.20285868144226837 +-1.3101852978323116 1:-0.3740821549570985 2:-0.9788976137554464 3:-0.6078739734947245 4:-0.8007745980271539 5:0.7381298546055934 6:0.7407750458109124 7:-0.7711351008178868 8:-0.9895256155202141 9:0.35793767138197174 10:0.6589909255086295 +0.5180809608973377 1:0.19289850282287446 2:0.6301214514538145 3:-0.15311307199521518 4:-0.8607670552113709 5:-0.46422067276745316 6:-0.29812862604449464 7:0.519464836430044 8:-0.9480450997338103 9:0.973503038633444 10:-0.7843880226794626 +1.9947872601406775 1:-0.15799682110486057 2:0.22645891561571352 3:0.3141842574216682 4:-0.36086019480721676 5:-0.1429373936064291 6:0.8097261636650581 7:0.11764088861630029 8:-0.9151998265501957 9:0.6536711690904891 10:-0.17232697113157425 +12.352290000973428 1:0.8176113135335772 2:0.39342616792621987 3:0.44011948797971234 4:-0.4412435869837865 5:-0.24509203724837314 6:0.8636655043434542 7:-0.4251583124505798 8:0.2068056615503988 9:-0.3501114760443049 10:-0.23701353324739483 +-2.891643319177732 1:0.7722403010820704 2:0.7994121584045861 3:0.18520464815273208 4:0.7273575609391227 5:-0.3758589216283552 6:-0.7598404862373955 7:0.5748649410179301 8:0.6897988099260968 9:0.5638920860629713 10:-0.992567809902162 +4.803737144054077 1:-0.7367711178556622 2:0.07370548192399351 3:-0.5510509754264419 4:0.11949095653894504 5:-0.7723751845800411 6:0.6450480728551136 7:-0.9508825019800493 8:-0.3250395411575804 9:-0.24913562167143777 10:-0.3617439870343031 +5.051689886526102 1:-0.09854955786627007 2:0.5298224514703289 3:-0.014996634675966236 4:-0.4462048687049027 5:0.22912790083984547 6:-0.513533454471272 7:0.1452771069237353 8:0.371152210841464 9:0.9204732090987018 10:0.7472990716905279 +3.8591142298280476 1:0.7532169023970261 2:0.8291433156934658 3:0.9255891263525324 4:0.3248663809949248 5:0.9905320652281553 6:-0.10383453745167626 7:0.8519246838852608 8:0.6024015353989258 9:-0.06958036249881938 10:0.5862142389541998 +11.30005914221598 1:0.026411858067972194 2:-0.6968445330429607 3:-0.8194566946165238 4:-0.12780659247925996 5:0.8406393783194903 6:-0.24617182945415128 7:0.30199973460219853 8:0.6062457235841974 9:-0.19314055910416927 10:-0.48313233883372964 +-10.288657252388708 1:-0.7388306404020344 2:0.07753617971873439 3:-0.5735498713988352 4:0.2183581175474576 5:-0.873572721679176 6:-0.8788755575751708 7:0.7087858362905568 8:0.7126712562404713 9:-0.7607334319316799 10:-0.4627367552114916 +4.895250842405817 1:0.9772954128558484 2:0.6020087399988574 3:0.16946626176056134 4:-0.011334492807484997 5:-0.5391845039589362 6:-0.4315843612118535 7:0.9065130011032458 8:-0.4860160207844919 9:0.0921755607946162 10:-0.022200673265013515 +1.0479421939727227 1:-0.055436367433274514 2:-0.6710483362647659 3:0.9222786043047919 4:-0.22005981623386184 5:-0.8141845044113469 6:-0.31766631447334226 7:0.6067696845798944 8:-0.1445661385071555 9:0.9172271611227454 10:-0.8079554780561127 +-9.754451457291598 1:0.533713237587885 2:0.6499588942067549 3:-0.49188790503368285 4:-0.6925119436487435 5:0.3345265979579788 6:-0.8117849521672496 7:0.9312055115656304 8:0.3273803451149724 9:0.7567478475677727 10:-0.6256676928549367 +5.869027126482974 1:0.7273823383600513 2:-0.2519813990388706 3:-0.8239584025397881 4:-0.13749750031735974 5:0.6142824732416132 6:0.6251630800232315 7:-0.6138240706157267 8:0.7210396245391326 9:-0.41832155201953714 10:-0.8965988320689853 +9.14234252751227 1:0.7295320896113133 2:0.6150271212503227 3:-0.9785024737101733 4:0.30006672036705506 5:0.11703528191771406 6:0.2971639460196238 7:-0.7920108995168815 8:0.32649036066184567 9:0.03522428067355543 10:-0.1766251898148803 +-5.643698771141404 1:0.27360638280623983 2:-0.6124401810442446 3:0.24950528730210886 4:0.09920211684887548 5:0.7187490549286091 6:0.6212724115415782 7:0.5864634211269566 8:0.114951165007104 9:0.44859258949094283 10:-0.3768352371578665 +12.781643819428492 1:0.9144335582094396 2:-0.4579872615218674 3:-0.6521934534632468 4:0.4462086111316512 5:0.240360283350179 6:0.23974046479581124 7:0.4840439971437822 8:-0.7250363120037027 9:-0.29769496257362094 10:-0.3382859512018359 +8.393556738722923 1:-0.8263387132502396 2:0.9434824094966923 3:0.1607861709872136 4:0.15217100448798782 5:-0.6517945935711484 6:-3.354731073326178E-4 7:0.07846631386981562 8:0.687844846942889 9:0.9277854407325892 10:-0.8855380268588307 +-15.348871155379253 1:-0.5734707274250155 2:-0.2526008551945753 3:0.23752094195309925 4:-0.7074613963298721 5:0.4674168537545218 6:-0.3198997855552628 7:-0.10415974108745596 8:0.5616912699671224 9:0.43742425558560694 10:0.19732530755184596 +13.138260063721448 1:-0.9415220143797984 2:0.6015431361268124 3:0.38898046240229545 4:-0.5750448371021175 5:-0.5803995196333898 6:0.11772198725731342 7:0.7512685244060366 8:-0.6683465740662857 9:0.9515652825318053 10:-0.32405935964523547 +-26.736207182601724 1:-0.47083104147202404 2:0.28748860067800597 3:0.007399318769021113 4:-0.8189013750589702 5:-0.5156633937248272 6:-0.9906928746525896 7:-0.8848419810272337 8:0.2197280161306785 9:0.12855082514870197 10:-0.7862803985146845 +-20.212077258958672 1:0.5609065808412279 2:-0.9201904391147984 3:0.908305865183735 4:0.9255146658282842 5:0.6871419344095282 6:0.4201876217923466 7:-0.42906289792612684 8:0.5787691868233418 9:0.7260522064761288 10:0.28251641556690554 +-0.44652227528840105 1:0.37640618494870504 2:-0.20012451052963542 3:0.9420894309510319 4:0.4218728633972739 5:0.5551974480349577 6:0.07615991810462619 7:-0.12409220462011294 8:-0.22212591926375946 9:0.21160498862483723 10:-0.6092792830633924 +-1.9481059746438067 1:-0.43820030250217457 2:-0.6836588417639442 3:0.733018205278934 4:-0.6564348753121718 5:0.7333385435136448 6:-0.5577457688360317 7:-0.31035811050608975 8:-0.7189201447768139 9:-0.7629842028723994 10:0.7179459779331092 +1.1951162998609508 1:0.19541555859727744 2:-0.4796785506546435 3:0.14123852670749248 4:0.7161847585887089 5:-0.2502765085719578 6:0.8815667909545981 7:-0.6418691905513725 8:0.49600147195728783 9:-0.3091837674381053 10:0.4320162841463153 +-8.99125390483227 1:-0.01183888602092864 2:-0.5901829024081027 3:-0.4343074406380647 4:-0.40450313056290166 5:0.05269590196351448 6:0.733631212862198 7:0.9575176715505025 8:0.5974628692830348 9:-0.20284241796038271 10:0.9577348510907686 +-7.955533026930219 1:0.6104830760481679 2:0.5915483572646505 3:0.3275427350991458 4:0.48361434056132424 5:-0.9466590639056058 6:-0.24662428438925743 7:0.9856361456534972 8:0.9434155212648045 9:0.3466736921968707 10:0.12927980558284102 +-12.500773785355054 1:0.5733321361720694 2:0.39154119830075085 3:-0.9347116355607772 4:0.0920586614926524 5:-0.6959457183810456 6:0.2136579936466858 7:0.17595268059814395 8:0.8828168055200465 9:0.18934277314853398 10:0.7565908584660754 +-11.43180236554046 1:0.082018621904135 2:0.9074181204118958 3:0.46125595008850273 4:0.40328845936169966 5:0.7803064691948824 6:0.20802011482729377 7:-0.41368899649077284 8:-0.8997565495498339 9:-0.1880483213318005 10:-0.15538597634233264 +-5.055293333055445 1:0.4442675297698402 2:0.19045719972922193 3:0.4877438951288897 4:0.7984474402420494 5:0.3251350777349489 6:-0.18676050499673869 7:-0.2701840041572374 8:0.4486609996458524 9:0.5403637876036615 10:-0.8971614841211264 +1.0276485382241776 1:0.7953696703382547 2:-0.3245779681908927 3:-0.3507435626548021 4:0.9510986059491036 5:-0.8655491074076527 6:0.20729233888498677 7:-0.43078300089533594 8:0.19504657032168216 9:-0.3173814102187291 10:-0.042479969052890754 +9.690201571311908 1:0.16852987139559206 2:-0.2514893273405625 3:-0.9993240281686275 4:-0.2166013247997891 5:0.33294165754921234 6:-0.5824203831560628 7:-0.15253642946648616 8:0.3547892367555441 9:-0.047604356104869794 10:0.9229112136183077 +2.2591036039970347 1:-0.9919593184325572 2:0.6323551392201245 3:-0.20815293136790447 4:-0.002395046469600759 5:-0.5015903362190326 6:-0.16698803749234048 7:0.7901657583805675 8:0.33755402936964973 9:-0.3707337678548108 10:0.6995480653730146 +1.5130881908855742 1:0.973710432688613 2:0.6518972988019702 3:-0.16491318496856833 4:-0.6066757853095415 5:0.8762371591845273 6:-0.9056066630820714 7:-0.3388079327070965 8:0.3934146060660142 9:-0.8756168865642253 10:0.9522427911640303 +4.023618949132531 1:-0.14974626191548301 2:-0.5874962377709136 3:0.6780439909311404 4:-0.37291203746764356 5:0.08104034602232169 6:-0.4706923395029945 7:-0.8924577368048239 8:-0.3363784341297067 9:-0.4139746050396018 10:-0.5107600309932907 +-2.8674162893420965 1:-0.7554383289076523 2:-0.1355597928418868 3:-0.3891904246986413 4:0.43949832438341785 5:-0.43859957095446833 6:0.37548094528561093 7:-0.5228633291549518 8:0.24169710795100352 9:0.7131753590746546 10:0.03458176767001042 +4.661164232198611 1:-0.12738868751385546 2:0.9446285809821182 3:-0.17981416859193433 4:-0.7535879975625193 5:-0.08594548726529161 6:-0.9983154486609989 7:-0.7272748852665216 8:-0.8197811039616518 9:0.5177610923333253 10:-0.6180731281817853 +-0.12347625601866746 1:0.10820547757674692 2:0.1825421454873002 3:-0.3412486258429426 4:-0.14925445930975534 5:-0.6594599831395103 6:0.9552502376248448 7:-0.7875626067291472 8:0.3854984181307912 9:0.014303876202374832 10:-0.7300443667550689 +14.546296184422973 1:0.2459523985646046 2:0.9434777073825811 3:0.2112745925235362 4:0.7730688005214974 5:-0.13727994893203732 6:0.6140037510172511 7:0.7545298281668846 8:0.7814551909982614 9:0.0026683642139069264 10:0.5633973602849358 +-19.66731861537172 1:0.9353590082406811 2:0.8768609458072838 3:0.9618210554140587 4:0.12103715737151921 5:-0.7691766106953688 6:-0.4220229608873225 7:-0.18117247651928658 8:-0.14333978019692784 9:-0.31512358142857066 10:0.4022153556528465 +18.84119697288412 1:0.4423204637505467 2:-0.4364821709544735 3:0.3935363893778452 4:-0.7750286735195999 5:-0.6981814766625978 6:0.6889512553826111 7:0.3646791168217727 8:0.0023536025493677837 9:-0.08378048150085249 10:-0.05659381771155503 +17.40329212914592 1:0.9155980216177384 2:-0.35593866074295355 3:0.44775710780914824 4:-0.42914421567532357 5:-0.2734430718503955 6:-0.8937042912745483 7:-0.3143761936611371 8:0.07805814979426184 9:-0.31386151509289784 10:0.6202932236456253 +-19.402336030214553 1:0.462288625222409 2:-0.902975525942725 3:0.7442695642729447 4:0.3802724233363486 5:0.4068685903786069 6:-0.5054707879424198 7:-0.8686166000900748 8:-0.014710838968344575 9:-0.1362606460134499 10:0.8444452252816472 +-3.855123203007599 1:0.5072557393175969 2:0.4626973233672753 3:-0.20910077161652119 4:0.9431415515135266 5:-0.1293690767585638 6:-0.2033835058111637 7:0.501429131658198 8:0.175133281735671 9:-0.6091682952201736 10:0.543010689352589 +1.493768355655548 1:-0.7772812666041105 2:-0.7743738591348672 3:-0.2848754060915175 4:0.3336846848765145 5:0.6219572132443736 6:-0.11144657683793624 7:0.7606913325884337 8:0.8547085151723017 9:-0.31728444617771134 10:-0.4668474022688931 +-17.803626188664516 1:0.5176340000264179 2:0.23048377874011128 3:0.6162746928601832 4:0.16908590014785418 5:0.9695207469685181 6:-0.34713218673384705 7:0.8526833760069625 8:0.9895592279649763 9:0.8805561957342884 10:-0.43452438291417894 +1.4060200157931342 1:-0.41964471941333525 2:0.7738486114171979 3:-0.0964606192284374 4:-0.25351781452566025 5:-0.21065389913054244 6:-0.40490416354122916 7:-0.7696501777959646 8:-0.7710488116813146 9:-0.6777228721053572 10:-0.09381158095961428 +-17.026492264209548 1:0.8367805314799452 2:0.1559190443625338 3:0.048200110551483544 4:-0.7340083467235765 5:0.2661150265782781 6:0.3881661781792165 7:0.9485287302765621 8:0.7201540574376382 9:0.8509234862656003 10:0.9658114866648093 +8.729450606651499 1:0.6404862166906327 2:0.16516090922657822 3:0.29013117743588057 4:0.37056732180613317 5:-0.3376494575302882 6:0.9012625630650577 7:-0.42150978319487 8:-0.05630249989686087 9:0.706104255632954 10:0.01935884085365225 +-5.516822117602276 1:-0.5718348423045241 2:-0.2145777722920088 3:-0.09307467998835195 4:-0.7311274103678378 5:0.5272184003067053 6:-0.00528176138162495 7:0.2852826178935919 8:0.6180999884045897 9:-0.7526372151008776 10:0.20416472532830543 +13.001541259752251 1:-0.5137703877272299 2:-0.15452359837207896 3:-0.25657600903152744 4:-0.9773110735601165 5:0.0718147980090178 6:0.18965211809311744 7:0.7795354990363292 8:0.21976898743223638 9:-0.20364089221752524 10:0.33822332985943304 +18.443388694564348 1:-0.9278344397401963 2:0.2678538727090136 3:-0.46932389854374734 4:0.8494176173177825 5:0.45765527018197694 6:0.20546395745879287 7:-0.199860294349123 8:0.47798730134403256 9:-0.2279771893187592 10:-0.30836118564314274 +8.952089112152663 1:-0.7371671220953286 2:0.8160149639986789 3:-0.026630089188139028 4:0.5931015267817183 5:-0.12216243475451294 6:0.161290795125286 7:0.7423016751095652 8:-0.5212872902985852 9:5.606147011660845E-5 10:-0.409626733921443 +-3.7062463981908027 1:0.5633514321449928 2:0.9914900963311462 3:0.1867799930236702 4:-0.15960235736142847 5:0.1204791067384241 6:-0.7733281422620872 7:-0.887447048141158 8:0.7931515335800692 9:0.732289882696125 10:-0.034992898370363124 +-10.58331129986813 1:0.6627003739767989 2:0.10688718810947728 3:-0.49230090744757216 4:0.8936580036513948 5:0.012227929286241057 6:-0.1442038886014838 7:0.9203452040795139 8:-0.20719832624131262 9:0.29561869366253335 10:-0.08597725084864649 +9.818996211259908 1:0.580133516885796 2:0.07422424429848573 3:0.33438634998226924 4:0.26054797992533696 5:-0.8771304726537796 6:-0.9710990591964794 7:-0.1869287393875041 8:-0.6167738073093247 9:0.34401921428837245 10:0.6737600514607418 +-11.87816749996684 1:-0.7193071334885193 2:0.5247127705364141 3:-0.02978727198197606 4:0.18353223007701058 5:0.40350110058596944 6:-0.36002841871228686 7:-0.20781535546501528 8:0.5517883176456557 9:-0.9938027872744732 10:0.6245061418135955 +-12.198096564661412 1:0.27542314155961156 2:0.3459734388741733 3:-0.38737776987446937 4:0.6244101669171684 5:-0.7801218302490938 6:0.20444733666197523 7:-0.5667599464182904 8:-0.9462131580071358 9:0.5576565405741785 10:-0.9307557040059242 +-3.6610413123521357 1:0.045569951437504086 2:0.32203961277046145 3:-0.04228927426053675 4:-0.9435304938416831 5:0.3750509710699601 6:0.21298970117620142 7:0.5491054691791977 8:0.33695088608872203 9:-0.9923500858828505 10:-0.6402707119893463 +3.782742149409224 1:0.7795250611996376 2:0.43296979846218275 3:-0.6481485005937841 4:0.3235717281667645 5:-0.8067382770768907 6:-0.06740397503468509 7:-0.2835017205434338 8:-0.5875853498478532 9:-0.25699561837680585 10:0.7813561594373908 +-17.065399625876015 1:-0.01772446594568744 2:0.563282914714494 3:0.14232420381013955 4:0.031667902604941345 5:-0.7815348482900619 6:0.3657733497576803 7:0.7208326162626688 8:-0.7863253120180662 9:0.3329194167867533 10:0.6175752945608013 +16.23248797654815 1:0.2615647748812251 2:-0.6631801348538622 3:0.6420349382574477 4:-0.31980528388089846 5:0.38021930887251365 6:-0.060298437830818896 7:-0.8911652782989568 8:0.3424617259589986 9:-0.8515350749364614 10:-0.42354709676980207 +-5.015963911416578 1:-0.07890564237014686 2:-0.09864377281008885 3:-0.13139943914680408 4:0.6610949669857866 5:0.06777579108221987 6:-0.26586245727222835 7:0.17443498956808612 8:-0.3129854922817781 9:-0.37913757211269505 10:0.7627186373372121 +22.647750304177556 1:-0.03666997412165163 2:0.49691867674483814 3:-0.45898559472166967 4:-0.09932248891016404 5:0.05692910907689508 6:-0.5006743461081364 7:0.9992936758550379 8:0.8252525466172065 9:0.9431711015127009 10:-0.4891497061921315 +-3.731112242951253 1:0.44353490207818513 2:0.23112032838224117 3:0.4697682541445527 4:-0.7507514828346664 5:-0.06323257550543837 6:0.0997091431243109 7:0.9394036761509628 8:0.4103869738859962 9:0.6564209227640914 10:-0.5427466755921158 +0.6761872737225261 1:-0.30051626190360503 2:-0.26699232020158803 3:0.8668758741279379 4:-0.40325291744583347 5:-0.9756425738484267 6:-0.5116398654634617 7:0.16424789009043073 8:0.8034099442414044 9:0.8554935001446193 10:0.42747702930667497 +8.449247195197387 1:-0.6588765973399024 2:0.2502285196526799 3:-0.20481547024283087 4:0.3770725284683252 5:-0.169707887761277 6:-0.0804075502584003 7:-0.3580757176408007 8:-0.6042549664471129 9:0.360349278976142 10:0.15899650901110962 +27.111027963108548 1:0.7106841652047162 2:0.6853699382312817 3:-0.8076297545289823 4:0.7932321056591545 5:-0.8011085095234463 6:-0.7017292726737878 7:0.10568649778064154 8:-0.40755358264969255 9:-0.061008981132773865 10:0.08895972651409556 +27.78383192005107 1:-0.8378790218922778 2:-0.6651002504721837 3:0.021049638665430415 4:0.32994334871293196 5:-0.7981304887988308 6:-0.2947962117284566 7:0.9739408711845776 8:0.9442893181893954 9:0.010541491359981059 10:0.8332791453382604 +15.700710963871254 1:-0.538773982400854 2:-0.5966426806845984 3:0.14570292467314627 4:-0.5937791901212952 5:0.7779251136963325 6:0.9962962075803357 7:-0.4774083823748394 8:-0.02528476957876369 9:-0.17305036341254398 10:-0.6013841506503688 +-12.558575788856189 1:0.03250364930617211 2:-0.6723950859659307 3:0.7090474884514901 4:0.25034305882632735 5:0.7036774024093582 6:0.622650236684523 7:0.5776881238206741 8:0.7999754726258337 9:0.21332972563833508 10:0.33849062947231645 +6.2776776518215955 1:-0.009605588630256623 2:0.5786496865369053 3:0.9208276908400748 4:-0.9477397424337148 5:0.6306053656362194 6:0.5396434662389846 7:-0.9841930450269964 8:0.5492682920407823 9:-0.020767248025529206 10:-0.8684655435686472 +6.424586997399564 1:0.861374923392324 2:0.8356037964367176 3:-0.7173479824827564 4:-0.6309584820438245 5:0.16136758138471285 6:-0.7485184163431866 7:-0.006053583829132236 8:-0.8762221084691306 9:0.19195377669247726 10:0.07259634302552964 +-9.64772485466405 1:0.7568015336230662 2:-0.4221524485756756 3:0.011711847664269248 4:0.7387065048724242 5:-0.04347512566745104 6:0.06642100869974654 7:-0.6993705848315939 8:0.16312217088045422 9:-0.11975577990989916 10:-0.6188717473788392 +3.8183706502283647 1:-0.7226937936463145 2:-0.5462756960199258 3:-0.39158419906610664 4:0.014310440945434433 5:-0.9950315917350652 6:-0.1844037449550875 7:0.9023517651879036 8:0.7948752060508435 9:-0.6792702010973877 10:0.40730074403235617 +1.1585019476700562 1:0.5575546848694 2:0.8997032130006739 3:0.6088643323129037 4:0.4872893656051758 5:-0.03977520372748922 6:0.3202565433572042 7:-0.31231768645537206 8:-0.6861153669592381 9:-0.08561643820383291 10:0.522243657731251 +-8.18651039877047 1:-0.809069379967462 2:-0.04827229852445103 3:0.19963602092982624 4:0.2568971171641006 5:-0.0015346733366310428 6:-0.6104625526166494 7:0.7746715041233412 8:-0.7343750018341593 9:-0.49272635466510106 10:-0.8115191199688623 +-3.377690136019927 1:-0.9408187510685164 2:0.9654993263332854 3:-0.16725010447984268 4:0.2574069587853294 5:-0.6930506968932861 6:0.11124762075550176 7:0.39145805505914866 8:0.2906495128462767 9:-0.27454907309824916 10:0.9001175309434777 +12.692571815413245 1:0.7404426710258791 2:0.9060576634778448 3:0.7023712021897308 4:-0.9808126157768493 5:0.03447666475715194 6:-0.4146339211599541 7:-0.7329651749553896 8:-0.2696019807317358 9:-0.9885367164723897 10:-0.8540304023043486 +2.5111054050889354 1:0.7448154454968356 2:-0.7532143233138027 3:-0.9724617436335079 4:0.662620399592766 5:0.45517204589358307 6:0.37409736074838684 7:0.337245076577648 8:0.50951903847353 9:0.2590369923587328 10:-0.3248257475117191 +-8.300340493749207 1:0.5504850435404609 2:0.5077232940244447 3:0.778859307357816 4:0.2601916883813373 5:-0.0032275666062382413 6:0.039752927221862855 7:0.19468432568826755 8:-0.2859531554546477 9:-0.4113477962970582 10:0.43272011953041667 +5.904938653193952 1:0.6622293273002955 2:0.6428891633785236 3:0.6999663090423285 4:0.9132698742913088 5:-0.3960072336866507 6:-0.14500922264286054 7:-0.4390171033743564 8:0.002067106212897185 9:-0.6079874251539117 10:-0.7131416109696531 +5.004048239623824 1:0.7212309895357449 2:0.3425199843383353 3:-0.7290323633040705 4:-0.5563097960397918 5:-0.7577898297822001 6:0.647883070472203 7:-0.23710559062843073 8:0.34398507133293954 9:-0.5440251617348038 10:-0.2971638032112218 +6.21255598077158 1:0.2498685983586959 2:-0.2586857335205359 3:-0.6380810501916263 4:0.17008841621855852 5:0.9485802018202867 6:-0.2580306792121272 7:0.032916516140567786 8:0.32950951532163675 9:-0.9291915084526683 10:0.8454021164786922 +-3.741044592262687 1:0.763300390779396 2:-0.1832552896771813 3:-0.39361907876758573 4:0.9050768615040607 5:-0.8850093869496836 6:0.9302208653737598 7:-0.12972094056755412 8:-0.459442486378308 9:0.5044112394875107 10:0.1399067554681861 +7.378402183384303 1:-0.27686808475610114 2:0.12735524561214606 3:0.5216635958678004 4:-0.9418584785460469 5:0.20441570818728771 6:-0.35073421178920583 7:0.7847501694079704 8:0.3222999552829353 9:0.21025696511089764 10:-0.5813710201294744 +-7.1500991588127265 1:-0.1945259148773102 2:-0.4089845159829022 3:-0.1971859124232922 4:0.9531447983295496 5:0.07996455700202221 6:0.17013529724757648 7:-0.2442095218739362 8:-0.8564146371721229 9:-0.5843910532907555 10:-0.33846471424918767 +-4.288417758202577 1:0.020710986120182184 2:-0.7450564238727908 3:0.3674992023059285 4:0.46737461414601555 5:0.9411702705113052 6:-0.7257365059912877 7:0.5813280037560231 8:-0.01567531846894843 9:0.24734195293533467 10:0.6516001002566887 +5.916426037500391 1:0.8260000862135342 2:-0.11324162495165968 3:0.13061304369435334 4:0.5762591624576425 5:0.548049763999644 6:-0.9751599851764361 7:0.02828821483057764 8:-0.4113286027346803 9:0.8912856976307486 10:-0.8470910204808244 +2.431004294471012 1:0.14088576701299083 2:-0.45104190898994734 3:0.29891134031619115 4:0.955503074037666 5:0.15962522624750242 6:0.7664481093046553 7:0.051697815479792686 8:-0.3471787155014081 9:-0.8007151537631465 10:-0.5598899500902301 +-16.08565904102149 1:0.3946137229565083 2:0.8443779319638349 3:0.5116855547320893 4:-0.5319339991982652 5:0.26564506849312797 6:0.18905397829944448 7:0.1976357098053687 8:0.15505612242632538 9:-0.935633748308776 10:-0.9782957013204887 +18.058440348477184 1:0.8402487524597533 2:-0.6200725197687718 3:-0.6158487677192792 4:0.0709328308135515 5:0.7501256905495493 6:0.38092209802839583 7:-0.8192579128383128 8:-0.9304002828581583 9:-0.6570300818845025 10:-0.5252554781538985 +-1.0026720160736349 1:0.46122079684901474 2:-0.7609201036934166 3:-0.9372178059537293 4:-0.25391036498391006 5:-0.7487429157699828 6:0.38024314675291637 7:0.21886059803198576 8:0.027516853267765207 9:0.33483464322377765 10:0.618580130027746 +-2.6688695419207162 1:-0.8775911623423445 2:-0.6647410420697879 3:0.05948516302547313 4:0.7278526664475804 5:-0.011366224409705028 6:0.33475665968289436 7:-0.6386120399761575 8:0.39609772177595115 9:-0.7872076290319412 10:-0.6195857302948329 +-13.867087895158768 1:-0.9114780602695882 2:0.7997695296649912 3:0.8337252417804881 4:-0.7927267913881113 5:0.6863829853181673 6:0.4162562153517635 7:0.2659922421074139 8:-0.551994669040742 9:-0.6403900338772157 10:-0.8680387717518072 +7.826011095515239 1:-0.2881951904396949 2:-0.19317071325391022 3:-0.06581062483451183 4:-0.6074074436315555 5:-0.9434740067975405 6:0.9426572655575483 7:-0.1812629432036228 8:0.39425575292939863 9:0.5065890539615039 10:0.8969825696966649 +1.4213836206303339 1:0.6996840540120932 2:0.1283999569152492 3:-0.2537375462472613 4:0.24772110606788456 5:0.9040210381745799 6:0.47062010977660207 7:0.9697678931927365 8:-0.9215764371674713 9:-0.27541598110075793 10:0.44277003247067803 +-0.973650798730175 1:-0.2121645467631068 2:-0.6770222508071349 3:-0.5733067523949165 4:0.27979529516037105 5:0.7128588235545461 6:-0.9208763636184307 7:0.14128337151047532 8:-0.002851660400375433 9:0.6943908711123281 10:-0.9201922993121072 +-0.17500848560451965 1:-0.6015070903427717 2:0.7815998200409671 3:-0.9932006200204946 4:-0.3303953411379028 5:-0.3329917860768894 6:-0.2822852019877604 7:0.6834785385197197 8:-0.6458607648553825 9:-0.06171476054995373 10:0.11421513352405444 +-15.310980589416289 1:-0.35290763483001486 2:-0.7263565311032778 3:-0.8688987069582226 4:-0.991098319894185 5:0.7029028082332363 6:-0.20251284356518684 7:-0.10928416773360117 8:0.307764663956116 9:0.6423143148384418 10:-0.15527637175127107 +3.260298266762908 1:-0.7817510582064782 2:0.45336200757318257 3:-0.15365670773321338 4:0.5063951567230205 5:-0.7102867196895872 6:-0.48050036620725955 7:0.9838016675169072 8:0.07854601230194436 9:-0.18953694857147863 10:0.19370072527454107 +3.846123583197846 1:0.6665586449040093 2:-0.2894063530813835 3:0.29965348483445386 4:0.23590344101670313 5:-0.7456743720187828 6:-0.4680876353446175 7:0.8106301610699425 8:0.691280702194663 9:-0.6060141408622055 10:0.34018639920235194 +-10.945919657782932 1:0.7669971723591666 2:0.38702771863552776 3:-0.6664311930513411 4:-0.2817072090916286 5:-0.16955916900934387 6:-0.9425831315444453 7:0.5685476711649924 8:-0.20782258743798265 9:0.015213591474494637 10:0.8183723865760859 +9.820049725467145 1:0.9582163993327679 2:0.7503905881505508 3:0.6255110430336392 4:0.6522701954798096 5:0.09248037700932144 6:-0.2833482854986902 7:-0.9841968940607242 8:-0.9343780716625845 9:-0.605526104070818 10:0.6000165028195326 +11.398715935456183 1:0.6605086903456443 2:0.14675454515266395 3:-0.7880053589830274 4:-0.8570785944515658 5:-0.4317693974151271 6:-0.12244918233307645 7:0.9808241653220866 8:0.5455853515046201 9:0.6870972425676756 10:0.7427686762232875 +-7.846310147695936 1:0.4355817642106965 2:0.7659504362110916 3:-0.3784171977305315 4:-0.5675896574776877 5:-0.20116390539973938 6:0.8775467546326667 7:-0.2824903364469842 8:0.7470660314619617 9:0.8967783051712528 10:0.7133700339519966 +-1.3847391232663768 1:0.3707613476850027 2:0.6931092598460797 3:-0.7701621508103305 4:-0.5679366502518555 5:-0.7234356749703683 6:-0.8059255104944509 7:-0.8307993875388229 8:0.6133975694770035 9:-0.7399749904168824 10:-0.1534990394513953 +16.93981662267873 1:0.6552665678625891 2:0.023248457840923775 3:-0.6850641408327465 4:0.7129790774369389 5:0.04166304042825364 6:-0.7160289667702797 7:-0.4733073680976494 8:0.2720897719417634 9:0.05850741911975099 10:0.34427554125371174 +2.8497179990245116 1:0.6664937514484015 2:0.3343796939204209 3:0.2611910348746209 4:-0.13658810351647 5:-0.5821801257591224 6:0.9854683468621908 7:-0.21396555404689188 8:-0.5923272173716836 9:-0.5674796199927252 10:-0.5681633547764235 +4.981807952389501 1:0.7517426071091595 2:0.7029291090701855 3:0.7126619831046563 4:-0.9982007415355478 5:-0.7743343367502893 6:-0.9048858749551119 7:-0.8243783842398396 8:0.4936163270697016 9:-0.6835495591484724 10:0.8412758607464845 +8.508637575729951 1:0.6837354268578517 2:-0.435346907350056 3:0.6597448795477736 4:0.8870204157376871 5:-0.6938576101541436 6:0.9199495715292882 7:0.33119640706964293 8:-0.6181273221979411 9:0.12929034268333317 10:0.6855150395247027 +14.369378079132883 1:-0.9489372180887643 2:-0.6577177233364067 3:0.543899463531252 4:0.5411152154119976 5:0.43733244485250733 6:0.5927084968109424 7:0.6100068837998656 8:0.9392735722529637 9:-0.9806701698603073 10:0.3984176141500082 +-6.456944198081549 1:0.8380442392342373 2:0.05166133486184443 3:-0.25864153418691704 4:-0.9506672344106888 5:0.5227275493542325 6:-0.03899736644563956 7:0.7660133053649136 8:-0.9375236703284806 9:-0.37213210747743175 10:0.0560768367274771 +-10.041353112580456 1:0.5293717914660876 2:-0.35874932480194044 3:0.14403824250820763 4:-0.4106496629336782 5:-0.794648717231762 6:-0.4369956159772408 7:0.8273613210141495 8:0.9212255384858874 9:0.00409867676727993 10:-0.23796544184855795 +-6.606325361718908 1:0.2765102732490652 2:0.10184669160432525 3:-0.9406443798496789 4:-0.46661976112717896 5:-0.5836573778289609 6:0.1308554421925976 7:0.05232199712543473 8:-0.4965370542771641 9:-0.3695836654343949 10:0.4874427445939513 +-15.359544879832677 1:-0.8253830145927283 2:0.29683545543963885 3:-0.9790356574071053 4:0.33749594518426473 5:-0.449483349548623 6:0.1740013774913005 7:0.5737323257916764 8:0.20159372721320645 9:-0.1812760896634873 10:-0.17652712339895738 +2.1801769966756845 1:0.3664130766917151 2:-0.1929450967547921 3:-0.7834945448457515 4:-0.03806442314852432 5:-0.6167622313628849 6:0.34919852301325394 7:-0.785891329691004 8:-0.5704062599527768 9:0.9846140894872721 10:-0.548571249100203 +-2.7006646885251415 1:-0.48505178676353067 2:0.06347121974094883 3:-0.3704723119141229 4:0.7407080276548548 5:0.06713252857406937 6:-0.2103524488773294 7:-0.9402467715192988 8:-0.8555624501612784 9:0.6244760190429901 10:-0.9038885681517279 +0.2105613019270259 1:-0.17125223509187282 2:-0.23328463772140529 3:-0.6497773470047024 4:0.33111604806115524 5:0.7944287248398398 6:0.5163977380074081 7:-0.025715995643062595 8:0.11762566041047462 9:0.9938658554834845 10:0.5363394203614278 +-0.6433952980357234 1:-0.905126800719938 2:0.5826442985002787 3:-0.8207546276288018 4:-0.0773547002692121 5:-0.6420058913410687 6:-0.9290787206193325 7:0.21829202840889095 8:-0.7752845890678082 9:0.4533233304372326 10:0.5457315861825041 +5.622874731146287 1:0.5486636398086722 2:-0.21867854114956642 3:0.13260110994566032 4:-0.024868470628895967 5:0.9246597814546305 6:0.07490395250443149 7:-0.21327567620097132 8:-0.33970581204395867 9:-0.19408398882121713 10:0.9757334811378136 +-18.27521356600463 1:-0.489685764918109 2:0.6832314342743568 3:0.9115808714640257 4:-4.680515344936964E-4 5:0.03760860984717218 6:0.4344127744883004 7:-0.30019645809377127 8:-0.48339658188341783 9:-0.5488933834939806 10:-0.4735052851773165 +5.518650144654079 1:-0.16881374315243192 2:0.22747702179774354 3:-0.8555270909193926 4:-0.6914231522703247 5:0.03618437407657238 6:-0.8404831131806643 7:0.16378525699004887 8:-0.333895928854854 9:0.23026574917978326 10:0.9409087845740918 +2.5599738684677646 1:-0.24371170373626905 2:-0.1752613047793694 3:-0.7930324885557696 4:0.17288443448968627 5:0.7233942014077801 6:0.47222694561171963 7:0.7878187692414558 8:-0.6520011755878357 9:-0.9952507460157223 10:-0.32951026378415094 +-8.508663400554862 1:0.9194236423060742 2:0.9517284917259223 3:-0.18723709334016392 4:-0.24913001260985546 5:0.8818286401027424 6:0.13661210218384512 7:-0.40792517201812983 8:-0.33132907984544957 9:-0.49137388288628703 10:-0.3273925353006979 +-10.233439586953153 1:0.0960128812383454 2:-0.8611756848964027 3:0.11807312551418647 4:-0.24570750746947145 5:-0.047826307143366886 6:-0.717269426008625 7:-0.2841658181308486 8:-0.31500935950449516 9:0.23183474949267713 10:-0.512986169560546 +-6.3459370724834265 1:0.9537835418930307 2:0.4598546399405288 3:-0.257013655072986 4:-0.29185820894937575 5:-0.6843688281544562 6:0.8343952028925479 7:-0.9656517094615942 8:-0.447440560943553 9:-0.9510349521362857 10:0.5918946980259567 +1.114406550703455 1:-0.5721838436595965 2:0.1201917297381252 3:-0.5253701290141362 4:-0.5874011312890843 5:0.7893580092022578 6:-0.18012813622584134 7:0.4781905737504004 8:-4.6732390143988667E-4 9:-0.7965374182885014 10:-0.8515444146742359 +8.688243146888663 1:0.2245581140502393 2:-0.0697600364101425 3:-0.7661833153629154 4:-0.2289151515902894 5:-0.5643191391300282 6:0.08069861795512168 7:-0.9670317635091523 8:0.14826752863715287 9:0.9325364047311011 10:0.4071178661803092 +14.896035572185347 1:0.20630949870309911 2:-0.5738578325975092 3:0.5664829389128903 4:0.3732752326637825 5:0.04079303403038881 6:-0.6604984910400766 7:0.15136076091734352 8:-0.6244939282579305 9:-0.5236288549540624 10:0.47284992666739023 +4.396558596072123 1:0.5565602414172521 2:0.1444095747909111 3:0.028227502879770272 4:0.38297378287943773 5:-0.26739745457451725 6:-0.708209627997985 7:0.7604483272526881 8:0.8072075261139096 9:0.11460574885028274 10:-0.07669406807610635 +1.7457141275341528 1:0.3668576517164046 2:-0.5352200081463954 3:0.5853385976871426 4:-0.4482551060006992 5:-0.5676795208498786 6:0.8043295590331514 7:-0.02160829797068753 8:0.42281303847010454 9:0.027894531623162466 10:-0.541120112980032 +-15.334767479922341 1:-0.036676500783341615 2:0.804758241454594 3:-0.0642091078911513 4:0.1402705435750966 5:-0.9215322030628859 6:0.7951173116514345 7:-0.994819896842561 8:0.2382406912119326 9:0.6634166177958731 10:0.7623222578718651 +5.017247792012723 1:-0.5925393497160352 2:0.48506599831456443 3:-0.5079795649118319 4:0.6668553329827696 5:-0.1103174867779837 6:0.7048535526809607 7:-0.9819230894106692 8:0.19609620625274982 9:0.5173985272313828 10:-0.11269849619148875 +6.201510810634532 1:-0.6802942101330738 2:0.898957584078176 3:0.853293387559251 4:0.6089336185656065 5:-0.9352626288322801 6:0.3208583332890447 7:-0.964481544931127 8:-0.8294773786068643 9:-0.8817311989413614 10:0.5165364663580934 +19.174935630244647 1:-0.20026105252200788 2:0.7276178994821614 3:0.7748716685190951 4:-0.7423420145576229 5:0.13147770471985032 6:-0.8382015712894606 7:0.021760992104270294 8:-0.24586987823702944 9:-0.05958177281299326 10:0.47347236224860834 +-14.822152909751189 1:0.7255660700197897 2:-0.22751988933383926 3:-0.08409197084114317 4:0.072750455428638 5:0.1841692073989072 6:-0.33838406658716513 7:-0.44701963574290526 8:0.5031210959133143 9:0.09640858549693743 10:0.9857351194637847 +-6.310082095945472 1:-0.7692076133438608 2:0.8533601511731044 3:0.676268298275629 4:-0.783895030001512 5:-0.8195462819549715 6:0.3963101354895673 7:-0.6254922461977397 8:-0.7521135990258581 9:-0.8032003997516024 10:0.8388672800826487 +8.853802632714807 1:0.46950948246522195 2:-0.6148693581037883 3:0.028739220735170656 4:-0.024281643566285815 5:-0.3495458137792231 6:-0.12347196435522867 7:0.5253894065203333 8:0.5100713458262918 9:0.63975795701667 10:0.08644353314625053 +-10.293714040655924 1:-0.17971950768550893 2:-0.6621720204354751 3:0.888036885802737 4:-0.04977483590350751 5:-0.8964991391283221 6:0.6873490822438724 7:0.42369087852118836 8:0.48972554317650663 9:0.8617233178519317 10:-0.8348331836605276 +0.23985611568891863 1:0.050526696983213215 2:0.8544297176525815 3:0.8586358519997579 4:-0.021299752441110487 5:0.2606696929560939 6:-0.39446486150105997 7:-0.4166234435381613 8:-0.6097643266459343 9:0.46633996256010146 10:-0.22521646199731027 +21.57719950299147 1:-0.5878491135126271 2:0.802134056970349 3:-0.5471017580843434 4:0.6067966843473331 5:-0.691712219323007 6:0.7814323754276735 7:0.31689445927290016 8:-0.1668780061940922 9:0.5285692389527452 10:0.8027091025203246 +-0.7836538830323514 1:0.5766794801558166 2:0.8281463568384935 3:0.5087453132796032 4:0.5212853344036532 5:0.6294700781054074 6:-0.9385097739886943 7:-0.13127371407538302 8:0.9845390503404141 9:-0.7224166213906742 10:-0.11155327354295896 +6.710413649604831 1:-0.6919803228062729 2:-0.6526904017578161 3:-0.34211291948607014 4:0.9094842803341618 5:-0.9454398661995895 6:0.3780766512494227 7:0.5823385348738088 8:0.8817830051841733 9:-0.514843382774189 10:0.32579701113259296 +5.384747201245483 1:-0.9661857672086316 2:-0.519769534339731 3:-0.4466396856529564 4:-0.4370113024678448 5:-0.6397400687811474 6:0.08225309277403725 7:-0.25936524603970756 8:-0.1711463274766858 9:-0.42848099098115755 10:-0.8096854737357237 +7.688509532916731 1:0.3892872094452817 2:-0.13306620868059982 3:-0.932974891205117 4:-0.8921357494146682 5:0.4806996560679244 6:-0.21500288444218696 7:-0.8911268070046585 8:-0.9510264953215406 9:0.1899740993687098 10:-0.43944320580463536 +2.2546997585565296 1:-0.5963883101717473 2:-0.01115153603404151 3:0.8781871380140298 4:0.7736250964135891 5:-0.7325745711528668 6:0.2518631794989008 7:0.5760249284318746 8:0.8690107952725199 9:0.02320853138646095 10:0.08570951531344417 +5.597710012706039 1:-0.5323512235815979 2:0.03366944321271936 3:0.5143537675853551 4:0.28471250955283445 5:0.4012202634439719 6:0.12032039285431151 7:-0.08108716844967812 8:0.30231384371011294 9:0.03259115565303028 10:0.9567467516929173 +-12.977848725392104 1:-0.5908891529017144 2:-0.7678208242918028 3:0.8512434510178621 4:-0.14910196410347298 5:0.6250260229199651 6:0.5393378705290228 7:-0.9573580597625002 8:-0.864881502860934 9:0.4175735160503429 10:0.4872169215922426 +10.35887243981476 1:-0.09126023790482862 2:0.18852634121926526 3:-0.13523918100503107 4:0.8333842692409983 5:-0.6015442103644761 6:0.5347736461652235 7:-0.823489760471118 8:0.5562688292037381 9:-0.807478561291906 10:-0.666881464988351 +0.4250502150408626 1:0.7771717566171905 2:-0.8729202752916785 3:-0.25782888805127024 4:-0.13605474993771205 5:0.5911781118120025 6:-0.8444023967853633 7:0.6787302541469229 8:-0.5444299313083194 9:0.356121883138657 10:-0.8845333845080687 +-0.8743487925900991 1:-0.9087681208947878 2:-0.292625136739453 3:-0.35113758823291774 4:-0.705933223571676 5:-0.6882289471031144 6:0.8350131255297044 7:-0.7659016065609232 8:0.11400114955653207 9:-0.9466143658505732 10:-0.5033643125229932 +-5.615143641864686 1:-0.6688289820084299 2:-0.4623159855015393 3:0.012827807007503855 4:-0.44521264878006117 5:-0.5563111031201406 6:-0.6065295981983794 7:0.3806712426786838 8:-0.11317152118817408 9:0.507896127467435 10:-0.8487801189674464 +-0.1829397047693725 1:0.09377558075225512 2:0.5774384503027374 3:-0.7104684187448009 4:-0.07285914169135976 5:-0.8797920488335114 6:0.6099615504974201 7:-0.8047440624324915 8:-0.6877856114263066 9:0.5843004021777447 10:0.5190581455348131 +18.479680552020344 1:0.9635517137863321 2:0.9954507816218203 3:0.11959899129360774 4:0.3753283274192787 5:-0.9386713095183621 6:0.0926833703812433 7:0.48003949462701323 8:0.9432769781973132 9:-0.9637036991931129 10:-0.4064407447273508 +1.3850645873427236 1:0.14476184437006356 2:-0.11280617018445871 3:-0.4385084538142101 4:-0.5961619435136434 5:0.419554626795412 6:-0.5047767472761191 7:0.457180284958592 8:-0.9129360314541999 9:-0.6320022059786656 10:-0.44989608519659363 diff --git a/data/mllib/sample_movielens_data.txt b/data/mllib/sample_movielens_data.txt new file mode 100644 index 0000000000000..f0eee19875f76 --- /dev/null +++ b/data/mllib/sample_movielens_data.txt @@ -0,0 +1,1501 @@ +0::2::3 +0::3::1 +0::5::2 +0::9::4 +0::11::1 +0::12::2 +0::15::1 +0::17::1 +0::19::1 +0::21::1 +0::23::1 +0::26::3 +0::27::1 +0::28::1 +0::29::1 +0::30::1 +0::31::1 +0::34::1 +0::37::1 +0::41::2 +0::44::1 +0::45::2 +0::46::1 +0::47::1 +0::48::1 +0::50::1 +0::51::1 +0::54::1 +0::55::1 +0::59::2 +0::61::2 +0::64::1 +0::67::1 +0::68::1 +0::69::1 +0::71::1 +0::72::1 +0::77::2 +0::79::1 +0::83::1 +0::87::1 +0::89::2 +0::91::3 +0::92::4 +0::94::1 +0::95::2 +0::96::1 +0::98::1 +0::99::1 +1::2::2 +1::3::1 +1::4::2 +1::6::1 +1::9::3 +1::12::1 +1::13::1 +1::14::1 +1::16::1 +1::19::1 +1::21::3 +1::27::1 +1::28::3 +1::33::1 +1::36::2 +1::37::1 +1::40::1 +1::41::2 +1::43::1 +1::44::1 +1::47::1 +1::50::1 +1::54::1 +1::56::2 +1::57::1 +1::58::1 +1::60::1 +1::62::4 +1::63::1 +1::67::1 +1::68::4 +1::70::2 +1::72::1 +1::73::1 +1::74::2 +1::76::1 +1::77::3 +1::78::1 +1::81::1 +1::82::1 +1::85::3 +1::86::2 +1::88::2 +1::91::1 +1::92::2 +1::93::1 +1::94::2 +1::96::1 +1::97::1 +2::4::3 +2::6::1 +2::8::5 +2::9::1 +2::10::1 +2::12::3 +2::13::1 +2::15::2 +2::18::2 +2::19::4 +2::22::1 +2::26::1 +2::28::1 +2::34::4 +2::35::1 +2::37::5 +2::38::1 +2::39::5 +2::40::4 +2::47::1 +2::50::1 +2::52::2 +2::54::1 +2::55::1 +2::57::2 +2::58::2 +2::59::1 +2::61::1 +2::62::1 +2::64::1 +2::65::1 +2::66::3 +2::68::1 +2::71::3 +2::76::1 +2::77::1 +2::78::1 +2::80::1 +2::83::5 +2::85::1 +2::87::2 +2::88::1 +2::89::4 +2::90::1 +2::92::4 +2::93::5 +3::0::1 +3::1::1 +3::2::1 +3::7::3 +3::8::3 +3::9::1 +3::14::1 +3::15::1 +3::16::1 +3::18::4 +3::19::1 +3::24::3 +3::26::1 +3::29::3 +3::33::1 +3::34::3 +3::35::1 +3::36::3 +3::37::1 +3::38::2 +3::43::1 +3::44::1 +3::46::1 +3::47::1 +3::51::5 +3::52::3 +3::56::1 +3::58::1 +3::60::3 +3::62::1 +3::65::2 +3::66::1 +3::67::1 +3::68::2 +3::70::1 +3::72::2 +3::76::3 +3::79::3 +3::80::4 +3::81::1 +3::83::1 +3::84::1 +3::86::1 +3::87::2 +3::88::4 +3::89::1 +3::91::1 +3::94::3 +4::1::1 +4::6::1 +4::8::1 +4::9::1 +4::10::1 +4::11::1 +4::12::1 +4::13::1 +4::14::2 +4::15::1 +4::17::1 +4::20::1 +4::22::1 +4::23::1 +4::24::1 +4::29::4 +4::30::1 +4::31::1 +4::34::1 +4::35::1 +4::36::1 +4::39::2 +4::40::3 +4::41::4 +4::43::2 +4::44::1 +4::45::1 +4::46::1 +4::47::1 +4::49::2 +4::50::1 +4::51::1 +4::52::4 +4::54::1 +4::55::1 +4::60::3 +4::61::1 +4::62::4 +4::63::3 +4::65::1 +4::67::2 +4::69::1 +4::70::4 +4::71::1 +4::73::1 +4::78::1 +4::84::1 +4::85::1 +4::87::3 +4::88::3 +4::89::2 +4::96::1 +4::97::1 +4::98::1 +4::99::1 +5::0::1 +5::1::1 +5::4::1 +5::5::1 +5::8::1 +5::9::3 +5::10::2 +5::13::3 +5::15::1 +5::19::1 +5::20::3 +5::21::2 +5::23::3 +5::27::1 +5::28::1 +5::29::1 +5::31::1 +5::36::3 +5::38::2 +5::39::1 +5::42::1 +5::48::3 +5::49::4 +5::50::3 +5::51::1 +5::52::1 +5::54::1 +5::55::5 +5::56::3 +5::58::1 +5::60::1 +5::61::1 +5::64::3 +5::65::2 +5::68::4 +5::70::1 +5::71::1 +5::72::1 +5::74::1 +5::79::1 +5::81::2 +5::84::1 +5::85::1 +5::86::1 +5::88::1 +5::90::4 +5::91::2 +5::95::2 +5::99::1 +6::0::1 +6::1::1 +6::2::3 +6::5::1 +6::6::1 +6::9::1 +6::10::1 +6::15::2 +6::16::2 +6::17::1 +6::18::1 +6::20::1 +6::21::1 +6::22::1 +6::24::1 +6::25::5 +6::26::1 +6::28::1 +6::30::1 +6::33::1 +6::38::1 +6::39::1 +6::43::4 +6::44::1 +6::45::1 +6::48::1 +6::49::1 +6::50::1 +6::53::1 +6::54::1 +6::55::1 +6::56::1 +6::58::4 +6::59::1 +6::60::1 +6::61::3 +6::63::3 +6::66::1 +6::67::3 +6::68::1 +6::69::1 +6::71::2 +6::73::1 +6::75::1 +6::77::1 +6::79::1 +6::81::1 +6::84::1 +6::85::3 +6::86::1 +6::87::1 +6::88::1 +6::89::1 +6::91::2 +6::94::1 +6::95::2 +6::96::1 +7::1::1 +7::2::2 +7::3::1 +7::4::1 +7::7::1 +7::10::1 +7::11::2 +7::14::2 +7::15::1 +7::16::1 +7::18::1 +7::21::1 +7::22::1 +7::23::1 +7::25::5 +7::26::1 +7::29::4 +7::30::1 +7::31::3 +7::32::1 +7::33::1 +7::35::1 +7::37::2 +7::39::3 +7::40::2 +7::42::2 +7::44::1 +7::45::2 +7::47::4 +7::48::1 +7::49::1 +7::53::1 +7::54::1 +7::55::1 +7::56::1 +7::59::1 +7::61::2 +7::62::3 +7::63::2 +7::66::1 +7::67::3 +7::74::1 +7::75::1 +7::76::3 +7::77::1 +7::81::1 +7::82::1 +7::84::2 +7::85::4 +7::86::1 +7::92::2 +7::96::1 +7::97::1 +7::98::1 +8::0::1 +8::2::4 +8::3::2 +8::4::2 +8::5::1 +8::7::1 +8::9::1 +8::11::1 +8::15::1 +8::18::1 +8::19::1 +8::21::1 +8::29::5 +8::31::3 +8::33::1 +8::35::1 +8::36::1 +8::40::2 +8::44::1 +8::45::1 +8::50::1 +8::51::1 +8::52::5 +8::53::5 +8::54::1 +8::55::1 +8::56::1 +8::58::4 +8::60::3 +8::62::4 +8::64::1 +8::67::3 +8::69::1 +8::71::1 +8::72::3 +8::77::3 +8::78::1 +8::79::1 +8::83::1 +8::85::5 +8::86::1 +8::88::1 +8::90::1 +8::92::2 +8::95::4 +8::96::3 +8::97::1 +8::98::1 +8::99::1 +9::2::3 +9::3::1 +9::4::1 +9::5::1 +9::6::1 +9::7::5 +9::9::1 +9::12::1 +9::14::3 +9::15::1 +9::19::1 +9::21::1 +9::22::1 +9::24::1 +9::25::1 +9::26::1 +9::30::3 +9::32::4 +9::35::2 +9::36::2 +9::37::2 +9::38::1 +9::39::1 +9::43::3 +9::49::5 +9::50::3 +9::53::1 +9::54::1 +9::58::1 +9::59::1 +9::60::1 +9::61::1 +9::63::3 +9::64::3 +9::68::1 +9::69::1 +9::70::3 +9::71::1 +9::73::2 +9::75::1 +9::77::2 +9::81::2 +9::82::1 +9::83::1 +9::84::1 +9::86::1 +9::87::4 +9::88::1 +9::90::3 +9::94::2 +9::95::3 +9::97::2 +9::98::1 +10::0::3 +10::2::4 +10::4::3 +10::7::1 +10::8::1 +10::10::1 +10::13::2 +10::14::1 +10::16::2 +10::17::1 +10::18::1 +10::21::1 +10::22::1 +10::24::1 +10::25::3 +10::28::1 +10::35::1 +10::36::1 +10::37::1 +10::38::1 +10::39::1 +10::40::4 +10::41::2 +10::42::3 +10::43::1 +10::49::3 +10::50::1 +10::51::1 +10::52::1 +10::55::2 +10::56::1 +10::58::1 +10::63::1 +10::66::1 +10::67::2 +10::68::1 +10::75::1 +10::77::1 +10::79::1 +10::86::1 +10::89::3 +10::90::1 +10::97::1 +10::98::1 +11::0::1 +11::6::2 +11::9::1 +11::10::1 +11::11::1 +11::12::1 +11::13::4 +11::16::1 +11::18::5 +11::19::4 +11::20::1 +11::21::1 +11::22::1 +11::23::5 +11::25::1 +11::27::5 +11::30::5 +11::32::5 +11::35::3 +11::36::2 +11::37::2 +11::38::4 +11::39::1 +11::40::1 +11::41::1 +11::43::2 +11::45::1 +11::47::1 +11::48::5 +11::50::4 +11::51::3 +11::59::1 +11::61::1 +11::62::1 +11::64::1 +11::66::4 +11::67::1 +11::69::5 +11::70::1 +11::71::3 +11::72::3 +11::75::3 +11::76::1 +11::77::1 +11::78::1 +11::79::5 +11::80::3 +11::81::4 +11::82::1 +11::86::1 +11::88::1 +11::89::1 +11::90::4 +11::94::2 +11::97::3 +11::99::1 +12::2::1 +12::4::1 +12::6::1 +12::7::3 +12::8::1 +12::14::1 +12::15::2 +12::16::4 +12::17::5 +12::18::2 +12::21::1 +12::22::2 +12::23::3 +12::24::1 +12::25::1 +12::27::5 +12::30::2 +12::31::4 +12::35::5 +12::38::1 +12::41::1 +12::44::2 +12::45::1 +12::50::4 +12::51::1 +12::52::1 +12::53::1 +12::54::1 +12::56::2 +12::57::1 +12::60::1 +12::63::1 +12::64::5 +12::66::3 +12::67::1 +12::70::1 +12::72::1 +12::74::1 +12::75::1 +12::77::1 +12::78::1 +12::79::3 +12::82::2 +12::83::1 +12::84::1 +12::85::1 +12::86::1 +12::87::1 +12::88::1 +12::91::3 +12::92::1 +12::94::4 +12::95::2 +12::96::1 +12::98::2 +13::0::1 +13::3::1 +13::4::2 +13::5::1 +13::6::1 +13::12::1 +13::14::2 +13::15::1 +13::17::1 +13::18::3 +13::20::1 +13::21::1 +13::22::1 +13::26::1 +13::27::1 +13::29::3 +13::31::1 +13::33::1 +13::40::2 +13::43::2 +13::44::1 +13::45::1 +13::49::1 +13::51::1 +13::52::2 +13::53::3 +13::54::1 +13::62::1 +13::63::2 +13::64::1 +13::68::1 +13::71::1 +13::72::3 +13::73::1 +13::74::3 +13::77::2 +13::78::1 +13::79::2 +13::83::3 +13::85::1 +13::86::1 +13::87::2 +13::88::2 +13::90::1 +13::93::4 +13::94::1 +13::98::1 +13::99::1 +14::1::1 +14::3::3 +14::4::1 +14::5::1 +14::6::1 +14::7::1 +14::9::1 +14::10::1 +14::11::1 +14::12::1 +14::13::1 +14::14::3 +14::15::1 +14::16::1 +14::17::1 +14::20::1 +14::21::1 +14::24::1 +14::25::2 +14::27::1 +14::28::1 +14::29::5 +14::31::3 +14::34::1 +14::36::1 +14::37::2 +14::39::2 +14::40::1 +14::44::1 +14::45::1 +14::47::3 +14::48::1 +14::49::1 +14::51::1 +14::52::5 +14::53::3 +14::54::1 +14::55::1 +14::56::1 +14::62::4 +14::63::5 +14::67::3 +14::68::1 +14::69::3 +14::71::1 +14::72::4 +14::73::1 +14::76::5 +14::79::1 +14::82::1 +14::83::1 +14::88::1 +14::93::3 +14::94::1 +14::95::2 +14::96::4 +14::98::1 +15::0::1 +15::1::4 +15::2::1 +15::5::2 +15::6::1 +15::7::1 +15::13::1 +15::14::1 +15::15::1 +15::17::2 +15::19::2 +15::22::2 +15::23::2 +15::25::1 +15::26::3 +15::27::1 +15::28::2 +15::29::1 +15::32::1 +15::33::2 +15::34::1 +15::35::2 +15::36::1 +15::37::1 +15::39::1 +15::42::1 +15::46::5 +15::48::2 +15::50::2 +15::51::1 +15::52::1 +15::58::1 +15::62::1 +15::64::3 +15::65::2 +15::72::1 +15::73::1 +15::74::1 +15::79::1 +15::80::1 +15::81::1 +15::82::2 +15::85::1 +15::87::1 +15::91::2 +15::96::1 +15::97::1 +15::98::3 +16::2::1 +16::5::3 +16::6::2 +16::7::1 +16::9::1 +16::12::1 +16::14::1 +16::15::1 +16::19::1 +16::21::2 +16::29::4 +16::30::2 +16::32::1 +16::34::1 +16::36::1 +16::38::1 +16::46::1 +16::47::3 +16::48::1 +16::49::1 +16::50::1 +16::51::5 +16::54::5 +16::55::1 +16::56::2 +16::57::1 +16::60::1 +16::63::2 +16::65::1 +16::67::1 +16::72::1 +16::74::1 +16::80::1 +16::81::1 +16::82::1 +16::85::5 +16::86::1 +16::90::5 +16::91::1 +16::93::1 +16::94::3 +16::95::2 +16::96::3 +16::98::3 +16::99::1 +17::2::1 +17::3::1 +17::6::1 +17::10::4 +17::11::1 +17::13::2 +17::17::5 +17::19::1 +17::20::5 +17::22::4 +17::28::1 +17::29::1 +17::33::1 +17::34::1 +17::35::2 +17::37::1 +17::38::1 +17::45::1 +17::46::5 +17::47::1 +17::49::3 +17::51::1 +17::55::5 +17::56::3 +17::57::1 +17::58::1 +17::59::1 +17::60::1 +17::63::1 +17::66::1 +17::68::4 +17::69::1 +17::70::1 +17::72::1 +17::73::3 +17::78::1 +17::79::1 +17::82::2 +17::84::1 +17::90::5 +17::91::3 +17::92::1 +17::93::1 +17::94::4 +17::95::2 +17::97::1 +18::1::1 +18::4::3 +18::5::2 +18::6::1 +18::7::1 +18::10::1 +18::11::4 +18::12::2 +18::13::1 +18::15::1 +18::18::1 +18::20::1 +18::21::2 +18::22::1 +18::23::2 +18::25::1 +18::26::1 +18::27::1 +18::28::5 +18::29::1 +18::31::1 +18::32::1 +18::36::1 +18::38::5 +18::39::5 +18::40::1 +18::42::1 +18::43::1 +18::44::4 +18::46::1 +18::47::1 +18::48::1 +18::51::2 +18::55::1 +18::56::1 +18::57::1 +18::62::1 +18::63::1 +18::66::3 +18::67::1 +18::70::1 +18::75::1 +18::76::3 +18::77::1 +18::80::3 +18::81::3 +18::82::1 +18::83::5 +18::84::1 +18::97::1 +18::98::1 +18::99::2 +19::0::1 +19::1::1 +19::2::1 +19::4::1 +19::6::2 +19::11::1 +19::12::1 +19::14::1 +19::23::1 +19::26::1 +19::31::1 +19::32::4 +19::33::1 +19::34::1 +19::37::1 +19::38::1 +19::41::1 +19::43::1 +19::45::1 +19::48::1 +19::49::1 +19::50::2 +19::53::2 +19::54::3 +19::55::1 +19::56::2 +19::58::1 +19::61::1 +19::62::1 +19::63::1 +19::64::1 +19::65::1 +19::69::2 +19::72::1 +19::74::3 +19::76::1 +19::78::1 +19::79::1 +19::81::1 +19::82::1 +19::84::1 +19::86::1 +19::87::2 +19::90::4 +19::93::1 +19::94::4 +19::95::2 +19::96::1 +19::98::4 +20::0::1 +20::1::1 +20::2::2 +20::4::2 +20::6::1 +20::8::1 +20::12::1 +20::21::2 +20::22::5 +20::24::2 +20::25::1 +20::26::1 +20::29::2 +20::30::2 +20::32::2 +20::39::1 +20::40::1 +20::41::2 +20::45::2 +20::48::1 +20::50::1 +20::51::3 +20::53::3 +20::55::1 +20::57::2 +20::60::1 +20::61::1 +20::64::1 +20::66::1 +20::70::2 +20::72::1 +20::73::2 +20::75::4 +20::76::1 +20::77::4 +20::78::1 +20::79::1 +20::84::2 +20::85::2 +20::88::3 +20::89::1 +20::90::3 +20::91::1 +20::92::2 +20::93::1 +20::94::4 +20::97::1 +21::0::1 +21::2::4 +21::3::1 +21::7::2 +21::11::1 +21::12::1 +21::13::1 +21::14::3 +21::17::1 +21::19::1 +21::20::1 +21::21::1 +21::22::1 +21::23::1 +21::24::1 +21::27::1 +21::29::5 +21::30::2 +21::38::1 +21::40::2 +21::43::3 +21::44::1 +21::45::1 +21::46::1 +21::48::1 +21::51::1 +21::53::5 +21::54::1 +21::55::1 +21::56::1 +21::58::3 +21::59::3 +21::64::1 +21::66::1 +21::68::1 +21::71::1 +21::73::1 +21::74::4 +21::80::1 +21::81::1 +21::83::1 +21::84::1 +21::85::3 +21::87::4 +21::89::2 +21::92::2 +21::96::3 +21::99::1 +22::0::1 +22::3::2 +22::5::2 +22::6::2 +22::9::1 +22::10::1 +22::11::1 +22::13::1 +22::14::1 +22::16::1 +22::18::3 +22::19::1 +22::22::5 +22::25::1 +22::26::1 +22::29::3 +22::30::5 +22::32::4 +22::33::1 +22::35::1 +22::36::3 +22::37::1 +22::40::1 +22::41::3 +22::44::1 +22::45::2 +22::48::1 +22::51::5 +22::55::1 +22::56::2 +22::60::3 +22::61::1 +22::62::4 +22::63::1 +22::65::1 +22::66::1 +22::68::4 +22::69::4 +22::70::3 +22::71::1 +22::74::5 +22::75::5 +22::78::1 +22::80::3 +22::81::1 +22::82::1 +22::84::1 +22::86::1 +22::87::3 +22::88::5 +22::90::2 +22::92::3 +22::95::2 +22::96::2 +22::98::4 +22::99::1 +23::0::1 +23::2::1 +23::4::1 +23::6::2 +23::10::4 +23::12::1 +23::13::4 +23::14::1 +23::15::1 +23::18::4 +23::22::2 +23::23::4 +23::24::1 +23::25::1 +23::26::1 +23::27::5 +23::28::1 +23::29::1 +23::30::4 +23::32::5 +23::33::2 +23::36::3 +23::37::1 +23::38::1 +23::39::1 +23::43::1 +23::48::5 +23::49::5 +23::50::4 +23::53::1 +23::55::5 +23::57::1 +23::59::1 +23::60::1 +23::61::1 +23::64::4 +23::65::5 +23::66::2 +23::67::1 +23::68::3 +23::69::1 +23::72::1 +23::73::3 +23::77::1 +23::82::2 +23::83::1 +23::84::1 +23::85::1 +23::87::3 +23::88::1 +23::95::2 +23::97::1 +24::4::1 +24::6::3 +24::7::1 +24::10::2 +24::12::1 +24::15::1 +24::19::1 +24::24::1 +24::27::3 +24::30::5 +24::31::1 +24::32::3 +24::33::1 +24::37::1 +24::39::1 +24::40::1 +24::42::1 +24::43::3 +24::45::2 +24::46::1 +24::47::1 +24::48::1 +24::49::1 +24::50::1 +24::52::5 +24::57::1 +24::59::4 +24::63::4 +24::65::1 +24::66::1 +24::67::1 +24::68::3 +24::69::5 +24::71::1 +24::72::4 +24::77::4 +24::78::1 +24::80::1 +24::82::1 +24::84::1 +24::86::1 +24::87::1 +24::88::2 +24::89::1 +24::90::5 +24::91::1 +24::92::1 +24::94::2 +24::95::1 +24::96::5 +24::98::1 +24::99::1 +25::1::3 +25::2::1 +25::7::1 +25::9::1 +25::12::3 +25::16::3 +25::17::1 +25::18::1 +25::20::1 +25::22::1 +25::23::1 +25::26::2 +25::29::1 +25::30::1 +25::31::2 +25::33::4 +25::34::3 +25::35::2 +25::36::1 +25::37::1 +25::40::1 +25::41::1 +25::43::1 +25::47::4 +25::50::1 +25::51::1 +25::53::1 +25::56::1 +25::58::2 +25::64::2 +25::67::2 +25::68::1 +25::70::1 +25::71::4 +25::73::1 +25::74::1 +25::76::1 +25::79::1 +25::82::1 +25::84::2 +25::85::1 +25::91::3 +25::92::1 +25::94::1 +25::95::1 +25::97::2 +26::0::1 +26::1::1 +26::2::1 +26::3::1 +26::4::4 +26::5::2 +26::6::3 +26::7::5 +26::13::3 +26::14::1 +26::16::1 +26::18::3 +26::20::1 +26::21::3 +26::22::5 +26::23::5 +26::24::5 +26::27::1 +26::31::1 +26::35::1 +26::36::4 +26::40::1 +26::44::1 +26::45::2 +26::47::1 +26::48::1 +26::49::3 +26::50::2 +26::52::1 +26::54::4 +26::55::1 +26::57::3 +26::58::1 +26::61::1 +26::62::2 +26::66::1 +26::68::4 +26::71::1 +26::73::4 +26::76::1 +26::81::3 +26::85::1 +26::86::3 +26::88::5 +26::91::1 +26::94::5 +26::95::1 +26::96::1 +26::97::1 +27::0::1 +27::9::1 +27::10::1 +27::18::4 +27::19::3 +27::20::1 +27::22::2 +27::24::2 +27::25::1 +27::27::3 +27::28::1 +27::29::1 +27::31::1 +27::33::3 +27::40::1 +27::42::1 +27::43::1 +27::44::3 +27::45::1 +27::51::3 +27::52::1 +27::55::3 +27::57::1 +27::59::1 +27::60::1 +27::61::1 +27::64::1 +27::66::3 +27::68::1 +27::70::1 +27::71::2 +27::72::1 +27::75::3 +27::78::1 +27::80::3 +27::82::1 +27::83::3 +27::86::1 +27::87::2 +27::90::1 +27::91::1 +27::92::1 +27::93::1 +27::94::2 +27::95::1 +27::98::1 +28::0::3 +28::1::1 +28::2::4 +28::3::1 +28::6::1 +28::7::1 +28::12::5 +28::13::2 +28::14::1 +28::15::1 +28::17::1 +28::19::3 +28::20::1 +28::23::3 +28::24::3 +28::27::1 +28::29::1 +28::33::1 +28::34::1 +28::36::1 +28::38::2 +28::39::2 +28::44::1 +28::45::1 +28::49::4 +28::50::1 +28::52::1 +28::54::1 +28::56::1 +28::57::3 +28::58::1 +28::59::1 +28::60::1 +28::62::3 +28::63::1 +28::65::1 +28::75::1 +28::78::1 +28::81::5 +28::82::4 +28::83::1 +28::85::1 +28::88::2 +28::89::4 +28::90::1 +28::92::5 +28::94::1 +28::95::2 +28::98::1 +28::99::1 +29::3::1 +29::4::1 +29::5::1 +29::7::2 +29::9::1 +29::10::3 +29::11::1 +29::13::3 +29::14::1 +29::15::1 +29::17::3 +29::19::3 +29::22::3 +29::23::4 +29::25::1 +29::29::1 +29::31::1 +29::32::4 +29::33::2 +29::36::2 +29::38::3 +29::39::1 +29::42::1 +29::46::5 +29::49::3 +29::51::2 +29::59::1 +29::61::1 +29::62::1 +29::67::1 +29::68::3 +29::69::1 +29::70::1 +29::74::1 +29::75::1 +29::79::2 +29::80::1 +29::81::2 +29::83::1 +29::85::1 +29::86::1 +29::90::4 +29::93::1 +29::94::4 +29::97::1 +29::99::1 diff --git a/dev/audit-release/audit_release.py b/dev/audit-release/audit_release.py index fa2f02dfecc75..230e900ecd4de 100755 --- a/dev/audit-release/audit_release.py +++ b/dev/audit-release/audit_release.py @@ -30,18 +30,18 @@ import time import urllib2 -## Fill in release details here: +# Fill in release details here: RELEASE_URL = "http://people.apache.org/~pwendell/spark-1.0.0-rc1/" RELEASE_KEY = "9E4FE3AF" RELEASE_REPOSITORY = "https://repository.apache.org/content/repositories/orgapachespark-1006/" RELEASE_VERSION = "1.0.0" SCALA_VERSION = "2.10.4" SCALA_BINARY_VERSION = "2.10" -## +# LOG_FILE_NAME = "spark_audit_%s" % time.strftime("%h_%m_%Y_%I_%M_%S") LOG_FILE = open(LOG_FILE_NAME, 'w') -WORK_DIR = "/tmp/audit_%s" % int(time.time()) +WORK_DIR = "/tmp/audit_%s" % int(time.time()) MAVEN_CMD = "mvn" GPG_CMD = "gpg" @@ -50,52 +50,63 @@ # Track failures failures = [] + def clean_work_files(): - print "OK to delete scratch directory '%s'? (y/N): " % WORK_DIR - response = raw_input() - if response == "y": - shutil.rmtree(WORK_DIR) - print "Should I delete the log output file '%s'? (y/N): " % LOG_FILE_NAME - response = raw_input() - if response == "y": - os.unlink(LOG_FILE_NAME) + print "OK to delete scratch directory '%s'? (y/N): " % WORK_DIR + response = raw_input() + if response == "y": + shutil.rmtree(WORK_DIR) + print "Should I delete the log output file '%s'? (y/N): " % LOG_FILE_NAME + response = raw_input() + if response == "y": + os.unlink(LOG_FILE_NAME) + def run_cmd(cmd, exit_on_failure=True): - print >> LOG_FILE, "Running command: %s" % cmd - ret = subprocess.call(cmd, shell=True, stdout=LOG_FILE, stderr=LOG_FILE) - if ret != 0 and exit_on_failure: - print "Command failed: %s" % cmd - clean_work_files() - sys.exit(-1) - return ret + print >> LOG_FILE, "Running command: %s" % cmd + ret = subprocess.call(cmd, shell=True, stdout=LOG_FILE, stderr=LOG_FILE) + if ret != 0 and exit_on_failure: + print "Command failed: %s" % cmd + clean_work_files() + sys.exit(-1) + return ret + def run_cmd_with_output(cmd): - print >> sys.stderr, "Running command: %s" % cmd - return subprocess.check_output(cmd, shell=True, stderr=LOG_FILE) + print >> sys.stderr, "Running command: %s" % cmd + return subprocess.check_output(cmd, shell=True, stderr=LOG_FILE) + def test(bool, str): - if bool: - return passed(str) - failed(str) + if bool: + return passed(str) + failed(str) + def passed(str): - print "[PASSED] %s" % str + print "[PASSED] %s" % str + def failed(str): - failures.append(str) - print "[**FAILED**] %s" % str + failures.append(str) + print "[**FAILED**] %s" % str + def get_url(url): - return urllib2.urlopen(url).read() + return urllib2.urlopen(url).read() + original_dir = os.getcwd() -# For each of these modules, we'll test an 'empty' application in sbt and +# For each of these modules, we'll test an 'empty' application in sbt and # maven that links against them. This will catch issues with messed up # dependencies within those projects. -modules = ["spark-core", "spark-bagel", "spark-mllib", "spark-streaming", "spark-repl", - "spark-graphx", "spark-streaming-flume", "spark-streaming-kafka", - "spark-streaming-mqtt", "spark-streaming-twitter", "spark-streaming-zeromq"] +modules = [ + "spark-core", "spark-bagel", "spark-mllib", "spark-streaming", "spark-repl", + "spark-graphx", "spark-streaming-flume", "spark-streaming-kafka", + "spark-streaming-mqtt", "spark-streaming-twitter", "spark-streaming-zeromq", + "spark-catalyst", "spark-sql", "spark-hive" +] modules = map(lambda m: "%s_%s" % (m, SCALA_BINARY_VERSION), modules) # Check for directories that might interfere with tests @@ -103,53 +114,57 @@ def get_url(url): cache_ivy_spark = "~/.ivy2/cache/org.apache.spark" local_maven_kafka = "~/.m2/repository/org/apache/kafka" local_maven_kafka = "~/.m2/repository/org/apache/spark" + + def ensure_path_not_present(x): - if os.path.exists(os.path.expanduser(x)): - print "Please remove %s, it can interfere with testing published artifacts." % x - sys.exit(-1) + if os.path.exists(os.path.expanduser(x)): + print "Please remove %s, it can interfere with testing published artifacts." % x + sys.exit(-1) + map(ensure_path_not_present, [local_ivy_spark, cache_ivy_spark, local_maven_kafka]) -# SBT build tests +# SBT build tests os.chdir("blank_sbt_build") os.environ["SPARK_VERSION"] = RELEASE_VERSION os.environ["SCALA_VERSION"] = SCALA_VERSION os.environ["SPARK_RELEASE_REPOSITORY"] = RELEASE_REPOSITORY +os.environ["SPARK_AUDIT_MASTER"] = "local" for module in modules: - os.environ["SPARK_MODULE"] = module - ret = run_cmd("sbt clean update", exit_on_failure=False) - test(ret == 0, "sbt build against '%s' module" % module) + os.environ["SPARK_MODULE"] = module + ret = run_cmd("sbt clean update", exit_on_failure=False) + test(ret == 0, "sbt build against '%s' module" % module) os.chdir(original_dir) # SBT application tests -for app in ["sbt_app_core", "sbt_app_graphx", "sbt_app_streaming"]: - os.chdir(app) - ret = run_cmd("sbt clean run", exit_on_failure=False) - test(ret == 0, "sbt application (%s)" % app) - os.chdir(original_dir) +for app in ["sbt_app_core", "sbt_app_graphx", "sbt_app_streaming", "sbt_app_sql", "sbt_app_hive"]: + os.chdir(app) + ret = run_cmd("sbt clean run", exit_on_failure=False) + test(ret == 0, "sbt application (%s)" % app) + os.chdir(original_dir) # Maven build tests os.chdir("blank_maven_build") for module in modules: - cmd = ('%s --update-snapshots -Dspark.release.repository="%s" -Dspark.version="%s" ' - '-Dspark.module="%s" clean compile' % - (MAVEN_CMD, RELEASE_REPOSITORY, RELEASE_VERSION, module)) - ret = run_cmd(cmd, exit_on_failure=False) - test(ret == 0, "maven build against '%s' module" % module) + cmd = ('%s --update-snapshots -Dspark.release.repository="%s" -Dspark.version="%s" ' + '-Dspark.module="%s" clean compile' % + (MAVEN_CMD, RELEASE_REPOSITORY, RELEASE_VERSION, module)) + ret = run_cmd(cmd, exit_on_failure=False) + test(ret == 0, "maven build against '%s' module" % module) os.chdir(original_dir) os.chdir("maven_app_core") mvn_exec_cmd = ('%s --update-snapshots -Dspark.release.repository="%s" -Dspark.version="%s" ' '-Dscala.binary.version="%s" clean compile ' - 'exec:java -Dexec.mainClass="SimpleApp"' % - (MAVEN_CMD, RELEASE_REPOSITORY, RELEASE_VERSION, SCALA_BINARY_VERSION)) + 'exec:java -Dexec.mainClass="SimpleApp"' % + (MAVEN_CMD, RELEASE_REPOSITORY, RELEASE_VERSION, SCALA_BINARY_VERSION)) ret = run_cmd(mvn_exec_cmd, exit_on_failure=False) test(ret == 0, "maven application (core)") os.chdir(original_dir) # Binary artifact tests if os.path.exists(WORK_DIR): - print "Working directory '%s' already exists" % WORK_DIR - sys.exit(-1) + print "Working directory '%s' already exists" % WORK_DIR + sys.exit(-1) os.mkdir(WORK_DIR) os.chdir(WORK_DIR) @@ -158,66 +173,66 @@ def ensure_path_not_present(x): artifacts = r.findall(index_page) for artifact in artifacts: - print "==== Verifying download integrity for artifact: %s ====" % artifact - - artifact_url = "%s/%s" % (RELEASE_URL, artifact) - run_cmd("wget %s" % artifact_url) - - key_file = "%s.asc" % artifact - run_cmd("wget %s/%s" % (RELEASE_URL, key_file)) - - run_cmd("wget %s%s" % (artifact_url, ".sha")) - - # Verify signature - run_cmd("%s --keyserver pgp.mit.edu --recv-key %s" % (GPG_CMD, RELEASE_KEY)) - run_cmd("%s %s" % (GPG_CMD, key_file)) - passed("Artifact signature verified.") - - # Verify md5 - my_md5 = run_cmd_with_output("%s --print-md MD5 %s" % (GPG_CMD, artifact)).strip() - release_md5 = get_url("%s.md5" % artifact_url).strip() - test(my_md5 == release_md5, "Artifact MD5 verified.") - - # Verify sha - my_sha = run_cmd_with_output("%s --print-md SHA512 %s" % (GPG_CMD, artifact)).strip() - release_sha = get_url("%s.sha" % artifact_url).strip() - test(my_sha == release_sha, "Artifact SHA verified.") - - # Verify Apache required files - dir_name = artifact.replace(".tgz", "") - run_cmd("tar xvzf %s" % artifact) - base_files = os.listdir(dir_name) - test("CHANGES.txt" in base_files, "Tarball contains CHANGES.txt file") - test("NOTICE" in base_files, "Tarball contains NOTICE file") - test("LICENSE" in base_files, "Tarball contains LICENSE file") - - os.chdir(WORK_DIR) - + print "==== Verifying download integrity for artifact: %s ====" % artifact + + artifact_url = "%s/%s" % (RELEASE_URL, artifact) + run_cmd("wget %s" % artifact_url) + + key_file = "%s.asc" % artifact + run_cmd("wget %s/%s" % (RELEASE_URL, key_file)) + + run_cmd("wget %s%s" % (artifact_url, ".sha")) + + # Verify signature + run_cmd("%s --keyserver pgp.mit.edu --recv-key %s" % (GPG_CMD, RELEASE_KEY)) + run_cmd("%s %s" % (GPG_CMD, key_file)) + passed("Artifact signature verified.") + + # Verify md5 + my_md5 = run_cmd_with_output("%s --print-md MD5 %s" % (GPG_CMD, artifact)).strip() + release_md5 = get_url("%s.md5" % artifact_url).strip() + test(my_md5 == release_md5, "Artifact MD5 verified.") + + # Verify sha + my_sha = run_cmd_with_output("%s --print-md SHA512 %s" % (GPG_CMD, artifact)).strip() + release_sha = get_url("%s.sha" % artifact_url).strip() + test(my_sha == release_sha, "Artifact SHA verified.") + + # Verify Apache required files + dir_name = artifact.replace(".tgz", "") + run_cmd("tar xvzf %s" % artifact) + base_files = os.listdir(dir_name) + test("CHANGES.txt" in base_files, "Tarball contains CHANGES.txt file") + test("NOTICE" in base_files, "Tarball contains NOTICE file") + test("LICENSE" in base_files, "Tarball contains LICENSE file") + + os.chdir(WORK_DIR) + for artifact in artifacts: - print "==== Verifying build and tests for artifact: %s ====" % artifact - os.chdir(os.path.join(WORK_DIR, dir_name)) - - os.environ["MAVEN_OPTS"] = "-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g" - # Verify build - print "==> Running build" - run_cmd("sbt assembly") - passed("sbt build successful") - run_cmd("%s package -DskipTests" % MAVEN_CMD) - passed("Maven build successful") - - # Verify tests - print "==> Performing unit tests" - run_cmd("%s test" % MAVEN_CMD) - passed("Tests successful") - os.chdir(WORK_DIR) + print "==== Verifying build and tests for artifact: %s ====" % artifact + os.chdir(os.path.join(WORK_DIR, dir_name)) + + os.environ["MAVEN_OPTS"] = "-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g" + # Verify build + print "==> Running build" + run_cmd("sbt assembly") + passed("sbt build successful") + run_cmd("%s package -DskipTests" % MAVEN_CMD) + passed("Maven build successful") + + # Verify tests + print "==> Performing unit tests" + run_cmd("%s test" % MAVEN_CMD) + passed("Tests successful") + os.chdir(WORK_DIR) clean_work_files() if len(failures) == 0: - print "ALL TESTS PASSED" + print "ALL TESTS PASSED" else: - print "SOME TESTS DID NOT PASS" - for f in failures: - print f + print "SOME TESTS DID NOT PASS" + for f in failures: + print f os.chdir(original_dir) diff --git a/dev/audit-release/maven_app_core/pom.xml b/dev/audit-release/maven_app_core/pom.xml index 0b837c01751fe..76a381f8e17e0 100644 --- a/dev/audit-release/maven_app_core/pom.xml +++ b/dev/audit-release/maven_app_core/pom.xml @@ -49,7 +49,7 @@ maven-compiler-plugin - 2.3.2 + 3.1 diff --git a/dev/audit-release/maven_app_core/src/main/java/SimpleApp.java b/dev/audit-release/maven_app_core/src/main/java/SimpleApp.java index 6b65dda39b1a2..5217689e7c092 100644 --- a/dev/audit-release/maven_app_core/src/main/java/SimpleApp.java +++ b/dev/audit-release/maven_app_core/src/main/java/SimpleApp.java @@ -37,5 +37,6 @@ public static void main(String[] args) { System.exit(-1); } System.out.println("Test succeeded"); + sc.stop(); } } diff --git a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala index 53fe43215e40e..77bbd167b199a 100644 --- a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala +++ b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala @@ -19,13 +19,18 @@ package main.scala import scala.util.Try +import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ object SimpleApp { def main(args: Array[String]) { + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple Spark App").setMaster(master) + case None => new SparkConf().setAppName("Simple Spark App") + } val logFile = "input.txt" - val sc = new SparkContext("local", "Simple App") + val sc = new SparkContext(conf) val logData = sc.textFile(logFile, 2).cache() val numAs = logData.filter(line => line.contains("a")).count() val numBs = logData.filter(line => line.contains("b")).count() diff --git a/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala b/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala index da08e014ebd94..24c7f8d667296 100644 --- a/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala +++ b/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala @@ -17,14 +17,20 @@ package main.scala -import org.apache.spark.SparkContext +import org.apache.spark.{SparkContext, SparkConf} import org.apache.spark.SparkContext._ import org.apache.spark.graphx._ import org.apache.spark.rdd.RDD object GraphXApp { def main(args: Array[String]) { - val sc = new SparkContext("local", "Simple GraphX App") + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple GraphX App").setMaster(master) + case None => new SparkConf().setAppName("Simple Graphx App") + } + val sc = new SparkContext(conf) + SparkContext.jarOfClass(this.getClass).foreach(sc.addJar) + val users: RDD[(VertexId, (String, String))] = sc.parallelize(Array((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")), (5L, ("franklin", "prof")), (2L, ("istoica", "prof")), diff --git a/dev/audit-release/sbt_app_hive/build.sbt b/dev/audit-release/sbt_app_hive/build.sbt new file mode 100644 index 0000000000000..7ac1be729c561 --- /dev/null +++ b/dev/audit-release/sbt_app_hive/build.sbt @@ -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. +// + +name := "Simple Project" + +version := "1.0" + +scalaVersion := System.getenv.get("SCALA_VERSION") + +libraryDependencies += "org.apache.spark" %% "spark-hive" % System.getenv.get("SPARK_VERSION") + +resolvers ++= Seq( + "Spark Release Repository" at System.getenv.get("SPARK_RELEASE_REPOSITORY"), + "Akka Repository" at "http://repo.akka.io/releases/", + "Spray Repository" at "http://repo.spray.cc/") diff --git a/dev/audit-release/sbt_app_hive/data.txt b/dev/audit-release/sbt_app_hive/data.txt new file mode 100644 index 0000000000000..0229e67f51e01 --- /dev/null +++ b/dev/audit-release/sbt_app_hive/data.txt @@ -0,0 +1,9 @@ +0val_0 +1val_1 +2val_2 +3val_3 +4val_4 +5val_5 +6val_6 +7val_7 +9val_9 diff --git a/dev/audit-release/sbt_app_hive/src/main/resources/hive-site.xml b/dev/audit-release/sbt_app_hive/src/main/resources/hive-site.xml new file mode 100644 index 0000000000000..93b835813d535 --- /dev/null +++ b/dev/audit-release/sbt_app_hive/src/main/resources/hive-site.xml @@ -0,0 +1,213 @@ + + + + + + + + + + + + + + + + + + build.dir + ${user.dir}/build + + + + build.dir.hive + ${build.dir}/hive + + + + hadoop.tmp.dir + ${build.dir.hive}/test/hadoop-${user.name} + A base for other temporary directories. + + + + + + hive.exec.scratchdir + ${build.dir}/scratchdir + Scratch space for Hive jobs + + + + hive.exec.local.scratchdir + ${build.dir}/localscratchdir/ + Local scratch space for Hive jobs + + + + javax.jdo.option.ConnectionURL + + jdbc:derby:;databaseName=../build/test/junit_metastore_db;create=true + + + + javax.jdo.option.ConnectionDriverName + org.apache.derby.jdbc.EmbeddedDriver + + + + javax.jdo.option.ConnectionUserName + APP + + + + javax.jdo.option.ConnectionPassword + mine + + + + + hive.metastore.warehouse.dir + ${test.warehouse.dir} + + + + + hive.metastore.metadb.dir + ${build.dir}/test/data/metadb/ + + Required by metastore server or if the uris argument below is not supplied + + + + + test.log.dir + ${build.dir}/test/logs + + + + + test.src.dir + ${build.dir}/src/test + + + + + + + hive.jar.path + ${build.dir.hive}/ql/hive-exec-${version}.jar + + + + + hive.metastore.rawstore.impl + org.apache.hadoop.hive.metastore.ObjectStore + Name of the class that implements org.apache.hadoop.hive.metastore.rawstore interface. This class is used to store and retrieval of raw metadata objects such as table, database + + + + hive.querylog.location + ${build.dir}/tmp + Location of the structured hive logs + + + + + + hive.task.progress + false + Track progress of a task + + + + hive.support.concurrency + false + Whether hive supports concurrency or not. A zookeeper instance must be up and running for the default hive lock manager to support read-write locks. + + + + fs.pfile.impl + org.apache.hadoop.fs.ProxyLocalFileSystem + A proxy for local file system used for cross file system testing + + + + hive.exec.mode.local.auto + false + + Let hive determine whether to run in local mode automatically + Disabling this for tests so that minimr is not affected + + + + + hive.auto.convert.join + false + Whether Hive enable the optimization about converting common join into mapjoin based on the input file size + + + + hive.ignore.mapjoin.hint + false + Whether Hive ignores the mapjoin hint + + + + hive.input.format + org.apache.hadoop.hive.ql.io.CombineHiveInputFormat + The default input format, if it is not specified, the system assigns it. It is set to HiveInputFormat for hadoop versions 17, 18 and 19, whereas it is set to CombineHiveInputFormat for hadoop 20. The user can always overwrite it - if there is a bug in CombineHiveInputFormat, it can always be manually set to HiveInputFormat. + + + + hive.default.rcfile.serde + org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + The default SerDe hive will use for the rcfile format + + + diff --git a/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala b/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala new file mode 100644 index 0000000000000..7257d17d10116 --- /dev/null +++ b/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.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 main.scala + +import scala.collection.mutable.{ListBuffer, Queue} + +import org.apache.spark.SparkConf +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.hive.LocalHiveContext + +case class Person(name: String, age: Int) + +object SparkSqlExample { + + def main(args: Array[String]) { + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple Sql App").setMaster(master) + case None => new SparkConf().setAppName("Simple Sql App") + } + val sc = new SparkContext(conf) + val hiveContext = new LocalHiveContext(sc) + + import hiveContext._ + hql("DROP TABLE IF EXISTS src") + hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") + hql("LOAD DATA LOCAL INPATH 'data.txt' INTO TABLE src") + val results = hql("FROM src SELECT key, value WHERE key >= 0 AND KEY < 5").collect() + results.foreach(println) + + def test(f: => Boolean, failureMsg: String) = { + if (!f) { + println(failureMsg) + System.exit(-1) + } + } + + test(results.size == 5, "Unexpected number of selected elements: " + results) + println("Test succeeded") + sc.stop() + } +} diff --git a/dev/audit-release/sbt_app_sql/build.sbt b/dev/audit-release/sbt_app_sql/build.sbt new file mode 100644 index 0000000000000..6e0ad3b4b2960 --- /dev/null +++ b/dev/audit-release/sbt_app_sql/build.sbt @@ -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. +// + +name := "Simple Project" + +version := "1.0" + +scalaVersion := System.getenv.get("SCALA_VERSION") + +libraryDependencies += "org.apache.spark" %% "spark-sql" % System.getenv.get("SPARK_VERSION") + +resolvers ++= Seq( + "Spark Release Repository" at System.getenv.get("SPARK_RELEASE_REPOSITORY"), + "Akka Repository" at "http://repo.akka.io/releases/", + "Spray Repository" at "http://repo.spray.cc/") 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 new file mode 100644 index 0000000000000..50af90c213b5a --- /dev/null +++ b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.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 main.scala + +import scala.collection.mutable.{ListBuffer, Queue} + +import org.apache.spark.SparkConf +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext + +case class Person(name: String, age: Int) + +object SparkSqlExample { + + def main(args: Array[String]) { + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple Sql App").setMaster(master) + case None => new SparkConf().setAppName("Simple Sql App") + } + val sc = new SparkContext(conf) + val sqlContext = new SQLContext(sc) + + import sqlContext._ + val people = sc.makeRDD(1 to 100, 10).map(x => Person(s"Name$x", x)) + people.registerAsTable("people") + val teenagers = sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") + val teenagerNames = teenagers.map(t => "Name: " + t(0)).collect() + teenagerNames.foreach(println) + + def test(f: => Boolean, failureMsg: String) = { + if (!f) { + println(failureMsg) + System.exit(-1) + } + } + + test(teenagerNames.size == 7, "Unexpected number of selected elements: " + teenagerNames) + println("Test succeeded") + sc.stop() + } +} diff --git a/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala b/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala index 3d0722d2ac45e..58a662bd9b2e8 100644 --- a/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala +++ b/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala @@ -27,9 +27,10 @@ import org.apache.spark.streaming._ object SparkStreamingExample { def main(args: Array[String]) { - val conf = new SparkConf(true) - .setMaster("local[2]") - .setAppName("Streaming test") + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple Streaming App").setMaster(master) + case None => new SparkConf().setAppName("Simple Streaming App") + } val ssc = new StreamingContext(conf, Seconds(1)) val seen = ListBuffer[RDD[Int]]() diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 995106f111443..c96afe822c6a9 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -21,47 +21,55 @@ # Publishes releases to Maven and packages/copies binary release artifacts. # Expects to be run in a totally empty directory. # +# Options: +# --package-only only packages an existing release candidate +# # Would be nice to add: # - Send output to stderr and have useful logging in stdout -# - Have this use sbt rather than Maven release plug in -GIT_USERNAME=pwendell -GIT_PASSWORD=XXX -GPG_PASSPHRASE=XXX -GIT_BRANCH=branch-0.9 -RELEASE_VERSION=0.9.0-incubating -RC_NAME=rc2 -USER_NAME=pwendell +GIT_USERNAME=${GIT_USERNAME:-pwendell} +GIT_PASSWORD=${GIT_PASSWORD:-XXX} +GPG_PASSPHRASE=${GPG_PASSPHRASE:-XXX} +GIT_BRANCH=${GIT_BRANCH:-branch-1.0} +RELEASE_VERSION=${RELEASE_VERSION:-1.0.0} +RC_NAME=${RC_NAME:-rc2} +USER_NAME=${USER_NAME:-pwendell} set -e -GIT_TAG=v$RELEASE_VERSION - -# Artifact publishing +GIT_TAG=v$RELEASE_VERSION-$RC_NAME -git clone https://git-wip-us.apache.org/repos/asf/spark.git -b $GIT_BRANCH -cd spark -export MAVEN_OPTS="-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g" +if [[ ! "$@" =~ --package-only ]]; then + echo "Creating and publishing release" + # Artifact publishing + git clone https://git-wip-us.apache.org/repos/asf/spark.git -b $GIT_BRANCH + cd spark + export MAVEN_OPTS="-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g" -mvn -Pyarn release:clean + mvn -Pyarn release:clean -mvn -DskipTests \ - -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ - -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ - -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Pspark-ganglia-lgpl \ - -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ - --batch-mode release:prepare + mvn -DskipTests \ + -Darguments="-DskipTests=true -Dmaven.javadoc.skip=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ + -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ + -Dmaven.javadoc.skip=true \ + -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ + -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ + -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ + --batch-mode release:prepare -mvn -DskipTests \ - -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ - -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Pspark-ganglia-lgpl\ - release:perform + mvn -DskipTests \ + -Darguments="-DskipTests=true -Dmaven.javadoc.skip=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ + -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ + -Dmaven.javadoc.skip=true \ + -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ + release:perform -rm -rf spark + cd .. + rm -rf spark +fi # Source and binary tarballs +echo "Packaging release tarballs" git clone https://git-wip-us.apache.org/repos/asf/spark.git cd spark git checkout --force $GIT_TAG @@ -83,16 +91,15 @@ rm -rf spark-$RELEASE_VERSION make_binary_release() { NAME=$1 - MAVEN_FLAGS=$2 - + FLAGS=$2 cp -r spark spark-$RELEASE_VERSION-bin-$NAME + cd spark-$RELEASE_VERSION-bin-$NAME - export MAVEN_OPTS="-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g" - mvn $MAVEN_FLAGS -DskipTests clean package - find . -name test-classes -type d | xargs rm -rf - find . -name classes -type d | xargs rm -rf + ./make-distribution.sh $FLAGS --name $NAME --tgz cd .. - tar cvzf spark-$RELEASE_VERSION-bin-$NAME.tgz spark-$RELEASE_VERSION-bin-$NAME + cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz . + rm -rf spark-$RELEASE_VERSION-bin-$NAME + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour \ --output spark-$RELEASE_VERSION-bin-$NAME.tgz.asc \ --detach-sig spark-$RELEASE_VERSION-bin-$NAME.tgz @@ -102,19 +109,18 @@ make_binary_release() { echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md \ SHA512 spark-$RELEASE_VERSION-bin-$NAME.tgz > \ spark-$RELEASE_VERSION-bin-$NAME.tgz.sha - rm -rf spark-$RELEASE_VERSION-bin-$NAME } -make_binary_release "hadoop1" "-Dhadoop.version=1.0.4" -make_binary_release "cdh4" "-Dhadoop.version=2.0.0-mr1-cdh4.2.0" -make_binary_release "hadoop2" "-Pyarn -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0" +make_binary_release "hadoop1" "--with-hive --hadoop 1.0.4" +make_binary_release "cdh4" "--with-hive --hadoop 2.0.0-mr1-cdh4.2.0" +make_binary_release "hadoop2" "--with-hive --with-yarn --hadoop 2.2.0" # Copy data echo "Copying release tarballs" -ssh $USER_NAME@people.apache.org \ - mkdir /home/$USER_NAME/public_html/spark-$RELEASE_VERSION-$RC_NAME rc_folder=spark-$RELEASE_VERSION-$RC_NAME -scp spark* \ +ssh $USER_NAME@people.apache.org \ + mkdir /home/$USER_NAME/public_html/$rc_folder +scp spark-* \ $USER_NAME@people.apache.org:/home/$USER_NAME/public_html/$rc_folder/ # Docs @@ -123,7 +129,9 @@ cd docs PRODUCTION=1 jekyll build echo "Copying release documentation" rc_docs_folder=${rc_folder}-docs -rsync -r _site/* $USER_NAME@people.apache.org /home/$USER_NAME/public_html/$rc_docs_folder +ssh $USER_NAME@people.apache.org \ + mkdir /home/$USER_NAME/public_html/$rc_docs_folder +rsync -r _site/* $USER_NAME@people.apache.org:/home/$USER_NAME/public_html/$rc_docs_folder echo "Release $RELEASE_VERSION completed:" echo "Git tag:\t $GIT_TAG" diff --git a/dev/create-release/generate-changelist.py b/dev/create-release/generate-changelist.py new file mode 100755 index 0000000000000..de1b5d4ae1314 --- /dev/null +++ b/dev/create-release/generate-changelist.py @@ -0,0 +1,148 @@ +#!/usr/bin/python + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# Creates CHANGES.txt from git history. +# +# Usage: +# First set the new release version and old CHANGES.txt version in this file. +# Make sure you have SPARK_HOME set. +# $ python generate-changelist.py + + +import os +import sys +import subprocess +import time +import traceback + +SPARK_HOME = os.environ["SPARK_HOME"] +NEW_RELEASE_VERSION = "1.0.0" +PREV_RELEASE_GIT_TAG = "v0.9.1" + +CHANGELIST = "CHANGES.txt" +OLD_CHANGELIST = "%s.old" % (CHANGELIST) +NEW_CHANGELIST = "%s.new" % (CHANGELIST) +TMP_CHANGELIST = "%s.tmp" % (CHANGELIST) + +# date before first PR in TLP Spark repo +SPARK_REPO_CHANGE_DATE1 = time.strptime("2014-02-26", "%Y-%m-%d") +# date after last PR in incubator Spark repo +SPARK_REPO_CHANGE_DATE2 = time.strptime("2014-03-01", "%Y-%m-%d") +# Threshold PR number that differentiates PRs to TLP +# and incubator repos +SPARK_REPO_PR_NUM_THRESH = 200 + +LOG_FILE_NAME = "changes_%s" % time.strftime("%h_%m_%Y_%I_%M_%S") +LOG_FILE = open(LOG_FILE_NAME, 'w') + + +def run_cmd(cmd): + try: + print >> LOG_FILE, "Running command: %s" % cmd + output = subprocess.check_output(cmd, shell=True, stderr=LOG_FILE) + print >> LOG_FILE, "Output: %s" % output + return output + except: + traceback.print_exc() + cleanup() + sys.exit(1) + + +def append_to_changelist(string): + with open(TMP_CHANGELIST, "a") as f: + print >> f, string + + +def cleanup(ask=True): + if ask is True: + print "OK to delete temporary and log files? (y/N): " + response = raw_input() + if ask is False or (ask is True and response == "y"): + if os.path.isfile(TMP_CHANGELIST): + os.remove(TMP_CHANGELIST) + if os.path.isfile(OLD_CHANGELIST): + os.remove(OLD_CHANGELIST) + LOG_FILE.close() + os.remove(LOG_FILE_NAME) + + +print "Generating new %s for Spark release %s" % (CHANGELIST, NEW_RELEASE_VERSION) +os.chdir(SPARK_HOME) +if os.path.isfile(TMP_CHANGELIST): + os.remove(TMP_CHANGELIST) +if os.path.isfile(OLD_CHANGELIST): + os.remove(OLD_CHANGELIST) + +append_to_changelist("Spark Change Log") +append_to_changelist("----------------") +append_to_changelist("") +append_to_changelist("Release %s" % NEW_RELEASE_VERSION) +append_to_changelist("") + +print "Getting commits between tag %s and HEAD" % PREV_RELEASE_GIT_TAG +hashes = run_cmd("git log %s..HEAD --pretty='%%h'" % PREV_RELEASE_GIT_TAG).split() + +print "Getting details of %s commits" % len(hashes) +for h in hashes: + date = run_cmd("git log %s -1 --pretty='%%ad' --date=iso | head -1" % h).strip() + subject = run_cmd("git log %s -1 --pretty='%%s' | head -1" % h).strip() + body = run_cmd("git log %s -1 --pretty='%%b'" % h) + committer = run_cmd("git log %s -1 --pretty='%%cn <%%ce>' | head -1" % h).strip() + body_lines = body.split("\n") + + if "Merge pull" in subject: + # Parse old format commit message + append_to_changelist(" %s %s" % (h, date)) + append_to_changelist(" %s" % subject) + append_to_changelist(" [%s]" % body_lines[0]) + append_to_changelist("") + + elif "maven-release" not in subject: + # Parse new format commit message + # Get authors from commit message, committer otherwise + authors = [committer] + if "Author:" in body: + authors = [line.split(":")[1].strip() for line in body_lines if "Author:" in line] + + # Generate GitHub PR URL for easy access if possible + github_url = "" + if "Closes #" in body: + pr_num = [line.split()[1].lstrip("#") for line in body_lines if "Closes #" in line][0] + github_url = "github.com/apache/spark/pull/%s" % pr_num + day = time.strptime(date.split()[0], "%Y-%m-%d") + if day < SPARK_REPO_CHANGE_DATE1 or + (day < SPARK_REPO_CHANGE_DATE2 and pr_num < SPARK_REPO_PR_NUM_THRESH): + github_url = "github.com/apache/incubator-spark/pull/%s" % pr_num + + append_to_changelist(" %s" % subject) + append_to_changelist(" %s" % ', '.join(authors)) + # for author in authors: + # append_to_changelist(" %s" % author) + append_to_changelist(" %s" % date) + if len(github_url) > 0: + append_to_changelist(" Commit: %s, %s" % (h, github_url)) + else: + append_to_changelist(" Commit: %s" % h) + append_to_changelist("") + +# Append old change list +print "Appending changelist from tag %s" % PREV_RELEASE_GIT_TAG +run_cmd("git show %s:%s | tail -n +3 >> %s" % (PREV_RELEASE_GIT_TAG, CHANGELIST, TMP_CHANGELIST)) +run_cmd("cp %s %s" % (TMP_CHANGELIST, NEW_CHANGELIST)) +print "New change list generated as %s" % NEW_CHANGELIST +cleanup(False) diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index e8f78fc5f231a..ffb70096d6014 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -1,4 +1,4 @@ -#!/usr/bin/python +#!/usr/bin/env python # # Licensed to the Apache Software Foundation (ASF) under one or more @@ -21,148 +21,263 @@ # usage: ./apache-pr-merge.py (see config env vars below) # # This utility assumes you already have local a Spark git folder and that you -# have added remotes corresponding to both (i) the github apache Spark +# have added remotes corresponding to both (i) the github apache Spark # mirror and (ii) the apache git repo. import json import os +import re import subprocess import sys import tempfile import urllib2 +try: + import jira.client + JIRA_IMPORTED = True +except ImportError: + JIRA_IMPORTED = False + # Location of your Spark git development area SPARK_HOME = os.environ.get("SPARK_HOME", "/home/patrick/Documents/spark") # Remote name which points to the Gihub site PR_REMOTE_NAME = os.environ.get("PR_REMOTE_NAME", "apache-github") # Remote name which points to Apache git PUSH_REMOTE_NAME = os.environ.get("PUSH_REMOTE_NAME", "apache") - -GIT_API_BASE = "https://api.github.com/repos/apache/spark" +# ASF JIRA username +JIRA_USERNAME = os.environ.get("JIRA_USERNAME", "pwendell") +# ASF JIRA password +JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", "1234") + +GITHUB_BASE = "https://github.com/apache/spark/pull" +GITHUB_API_BASE = "https://api.github.com/repos/apache/spark" +JIRA_BASE = "https://issues.apache.org/jira/browse" +JIRA_API_BASE = "https://issues.apache.org/jira" # Prefix added to temporary branches BRANCH_PREFIX = "PR_TOOL" os.chdir(SPARK_HOME) + def get_json(url): - try: - return json.load(urllib2.urlopen(url)) - except urllib2.HTTPError as e: - print "Unable to fetch URL, exiting: %s" % url - sys.exit(-1) + try: + return json.load(urllib2.urlopen(url)) + except urllib2.HTTPError as e: + print "Unable to fetch URL, exiting: %s" % url + sys.exit(-1) + def fail(msg): - print msg - clean_up() - sys.exit(-1) + print msg + clean_up() + sys.exit(-1) + def run_cmd(cmd): - if isinstance(cmd, list): - return subprocess.check_output(cmd) - else: - return subprocess.check_output(cmd.split(" ")) + if isinstance(cmd, list): + return subprocess.check_output(cmd) + else: + return subprocess.check_output(cmd.split(" ")) + def continue_maybe(prompt): - result = raw_input("\n%s (y/n): " % prompt) - if result.lower() != "y": - fail("Okay, exiting") + result = raw_input("\n%s (y/n): " % 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) + print "Restoring head pointer to %s" % original_head + run_cmd("git checkout %s" % original_head) + + branches = run_cmd("git branch").replace(" ", "").split("\n") - branches = run_cmd("git branch").replace(" ", "").split("\n") + for branch in filter(lambda x: x.startswith(BRANCH_PREFIX), branches): + print "Deleting local branch %s" % branch + run_cmd("git branch -D %s" % branch) - for branch in filter(lambda x: x.startswith(BRANCH_PREFIX), branches): - print "Deleting local branch %s" % branch - run_cmd("git branch -D %s" % branch) # merge the requested PR and return the merge hash def merge_pr(pr_num, target_ref): - 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)) - run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, target_ref, target_branch_name)) - run_cmd("git checkout %s" % target_branch_name) - - run_cmd(['git', 'merge', pr_branch_name, '--squash']) - - commit_authors = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, - '--pretty=format:%an <%ae>']).split("\n") - distinct_authors = sorted(set(commit_authors), key=lambda x: commit_authors.count(x), reverse=True) - primary_author = distinct_authors[0] - commits = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, - '--pretty=format:%h [%an] %s']).split("\n\n") - - merge_message_flags = [] + 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)) + run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, target_ref, target_branch_name)) + run_cmd("git checkout %s" % target_branch_name) + + had_conflicts = False + try: + run_cmd(['git', 'merge', pr_branch_name, '--squash']) + except Exception as e: + msg = "Error merging: %s\nWould you like to manually fix-up this merge?" % e + continue_maybe(msg) + msg = "Okay, please fix any conflicts and 'git add' conflicting files... Finished?" + continue_maybe(msg) + had_conflicts = True + + commit_authors = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, + '--pretty=format:%an <%ae>']).split("\n") + distinct_authors = sorted(set(commit_authors), + key=lambda x: commit_authors.count(x), reverse=True) + primary_author = distinct_authors[0] + commits = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, + '--pretty=format:%h [%an] %s']).split("\n\n") + + merge_message_flags = [] + + merge_message_flags += ["-m", title] + if body != None: + merge_message_flags += ["-m", body] + + authors = "\n".join(["Author: %s" % a for a in distinct_authors]) + + merge_message_flags += ["-m", authors] + + if had_conflicts: + committer_name = run_cmd("git config --get user.name").strip() + committer_email = run_cmd("git config --get user.email").strip() + message = "This patch had conflicts when merged, resolved by\nCommitter: %s <%s>" % ( + committer_name, committer_email) + merge_message_flags += ["-m", message] + + # The string "Closes #%s" string is required for GitHub to correctly close the PR + merge_message_flags += [ + "-m", + "Closes #%s from %s and squashes the following commits:" % (pr_num, pr_repo_desc)] + for c in commits: + merge_message_flags += ["-m", c] + + run_cmd(['git', 'commit', '--author="%s"' % primary_author] + merge_message_flags) + + continue_maybe("Merge complete (local ref %s). Push to %s?" % ( + target_branch_name, PUSH_REMOTE_NAME)) + + try: + run_cmd('git push %s %s:%s' % (PUSH_REMOTE_NAME, target_branch_name, target_ref)) + except Exception as e: + clean_up() + fail("Exception while pushing: %s" % e) + + merge_hash = run_cmd("git rev-parse %s" % target_branch_name)[:8] + clean_up() + print("Pull request #%s merged!" % pr_num) + print("Merge hash: %s" % merge_hash) + return merge_hash - for p in [title, body]: - merge_message_flags += ["-m", p] - authors = "\n".join(["Author: %s" % a for a in distinct_authors]) +def cherry_pick(pr_num, merge_hash, default_branch): + pick_ref = raw_input("Enter a branch name [%s]: " % default_branch) + if pick_ref == "": + pick_ref = default_branch - merge_message_flags += ["-m", authors] + pick_branch_name = "%s_PICK_PR_%s_%s" % (BRANCH_PREFIX, pr_num, pick_ref.upper()) - # The string "Closes #%s" string is required for GitHub to correctly close the PR - merge_message_flags += ["-m", - "Closes #%s from %s and squashes the following commits:" % (pr_num, pr_repo_desc)] - for c in commits: - merge_message_flags += ["-m", c] + run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, pick_ref, pick_branch_name)) + run_cmd("git checkout %s" % pick_branch_name) + run_cmd("git cherry-pick -sx %s" % merge_hash) - run_cmd(['git', 'commit', '--author="%s"' % primary_author] + merge_message_flags) + continue_maybe("Pick complete (local ref %s). Push to %s?" % ( + pick_branch_name, PUSH_REMOTE_NAME)) - continue_maybe("Merge complete (local ref %s). Push to %s?" % ( - target_branch_name, PUSH_REMOTE_NAME)) + try: + run_cmd('git push %s %s:%s' % (PUSH_REMOTE_NAME, pick_branch_name, pick_ref)) + except Exception as e: + clean_up() + fail("Exception while pushing: %s" % e) - try: - run_cmd('git push %s %s:%s' % (PUSH_REMOTE_NAME, target_branch_name, target_ref)) - except Exception as e: + pick_hash = run_cmd("git rev-parse %s" % pick_branch_name)[:8] clean_up() - fail("Exception while pushing: %s" % e) - - merge_hash = run_cmd("git rev-parse %s" % target_branch_name)[:8] - clean_up() - print("Pull request #%s merged!" % pr_num) - print("Merge hash: %s" % merge_hash) - return merge_hash - - -def maybe_cherry_pick(pr_num, merge_hash, default_branch): - continue_maybe("Would you like to pick %s into another branch?" % merge_hash) - pick_ref = raw_input("Enter a branch name [%s]: " % default_branch) - if pick_ref == "": - pick_ref = default_branch - - pick_branch_name = "%s_PICK_PR_%s_%s" % (BRANCH_PREFIX, pr_num, pick_ref.upper()) - - run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, pick_ref, pick_branch_name)) - run_cmd("git checkout %s" % pick_branch_name) - run_cmd("git cherry-pick -sx %s" % merge_hash) - - continue_maybe("Pick complete (local ref %s). Push to %s?" % ( - pick_branch_name, PUSH_REMOTE_NAME)) - - try: - run_cmd('git push %s %s:%s' % (PUSH_REMOTE_NAME, pick_branch_name, pick_ref)) - except Exception as e: - clean_up() - fail("Exception while pushing: %s" % e) - - pick_hash = run_cmd("git rev-parse %s" % pick_branch_name)[:8] - clean_up() - - print("Pull request #%s picked into %s!" % (pr_num, pick_ref)) - print("Pick hash: %s" % pick_hash) -branches = get_json("%s/branches" % GIT_API_BASE) + print("Pull request #%s picked into %s!" % (pr_num, pick_ref)) + print("Pick hash: %s" % pick_hash) + return pick_ref + + +def fix_version_from_branch(branch, versions): + # Note: Assumes this is a sorted (newest->oldest) list of un-released versions + if branch == "master": + return versions[0] + else: + branch_ver = branch.replace("branch-", "") + return filter(lambda x: x.name.startswith(branch_ver), versions)[-1] + + +def resolve_jira(title, merge_branches, comment): + asf_jira = jira.client.JIRA({'server': JIRA_API_BASE}, + basic_auth=(JIRA_USERNAME, JIRA_PASSWORD)) + + default_jira_id = "" + search = re.findall("SPARK-[0-9]{4,5}", title) + if len(search) > 0: + default_jira_id = search[0] + + jira_id = raw_input("Enter a JIRA id [%s]: " % default_jira_id) + if jira_id == "": + jira_id = default_jira_id + + try: + issue = asf_jira.issue(jira_id) + except Exception as e: + fail("ASF JIRA could not find %s\n%s" % (jira_id, e)) + + cur_status = issue.fields.status.name + cur_summary = issue.fields.summary + cur_assignee = issue.fields.assignee + if cur_assignee is None: + cur_assignee = "NOT ASSIGNED!!!" + else: + cur_assignee = cur_assignee.displayName + + if cur_status == "Resolved" or cur_status == "Closed": + fail("JIRA issue %s already has status '%s'" % (jira_id, cur_status)) + print ("=== JIRA %s ===" % jira_id) + print ("summary\t\t%s\nassignee\t%s\nstatus\t\t%s\nurl\t\t%s/%s\n" % ( + cur_summary, cur_assignee, cur_status, JIRA_BASE, jira_id)) + + versions = asf_jira.project_versions("SPARK") + versions = sorted(versions, key=lambda x: x.name, reverse=True) + versions = filter(lambda x: x.raw['released'] is False, versions) + + default_fix_versions = map(lambda x: fix_version_from_branch(x, versions).name, merge_branches) + for v in default_fix_versions: + # Handles the case where we have forked a release branch but not yet made the release. + # In this case, if the PR is committed to the master branch and the release branch, we + # only consider the release branch to be the fix version. E.g. it is not valid to have + # both 1.1.0 and 1.0.0 as fix versions. + (major, minor, patch) = v.split(".") + if patch == "0": + previous = "%s.%s.%s" % (major, int(minor) - 1, 0) + if previous in default_fix_versions: + default_fix_versions = filter(lambda x: x != v, default_fix_versions) + default_fix_versions = ",".join(default_fix_versions) + + fix_versions = raw_input("Enter comma-separated fix version(s) [%s]: " % default_fix_versions) + if fix_versions == "": + fix_versions = default_fix_versions + fix_versions = fix_versions.replace(" ", "").split(",") + + def get_version_json(version_str): + return filter(lambda v: v.name == version_str, versions)[0].raw + + jira_fix_versions = map(lambda v: get_version_json(v), fix_versions) + + resolve = filter(lambda a: a['name'] == "Resolve Issue", asf_jira.transitions(jira_id))[0] + 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) + + +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" % (GIT_API_BASE, pr_num)) +pr = get_json("%s/pulls/%s" % (GITHUB_API_BASE, pr_num)) url = pr["url"] title = pr["title"] @@ -172,29 +287,43 @@ def maybe_cherry_pick(pr_num, merge_hash, default_branch): base_ref = pr["head"]["ref"] pr_repo_desc = "%s/%s" % (user_login, base_ref) -if pr["merged"] == True: - print "Pull request %s has already been merged, assuming you want to backport" % pr_num - merge_commit_desc = run_cmd(['git', 'log', '--merges', '--first-parent', - '--grep=pull request #%s' % pr_num, '--oneline']).split("\n")[0] - if merge_commit_desc == "": - fail("Couldn't find any merge commit for #%s, you may need to update HEAD." % pr_num) +if pr["merged"] is True: + print "Pull request %s has already been merged, assuming you want to backport" % pr_num + merge_commit_desc = run_cmd([ + 'git', 'log', '--merges', '--first-parent', + '--grep=pull request #%s' % pr_num, '--oneline']).split("\n")[0] + if merge_commit_desc == "": + fail("Couldn't find any merge commit for #%s, you may need to update HEAD." % pr_num) - merge_hash = merge_commit_desc[:7] - message = merge_commit_desc[8:] - - print "Found: %s" % message - maybe_cherry_pick(pr_num, merge_hash, latest_branch) - sys.exit(0) + merge_hash = merge_commit_desc[:7] + message = merge_commit_desc[8:] -if bool(pr["mergeable"]) == False: - fail("Pull request %s is not mergeable in its current form" % pr_num) + print "Found: %s" % message + maybe_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) 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)) +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] + merge_hash = merge_pr(pr_num, target_ref) -while True: - maybe_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: + 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(title, merged_refs, jira_comment) +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." diff --git a/dev/mima b/dev/mima new file mode 100755 index 0000000000000..b68800d6d0173 --- /dev/null +++ b/dev/mima @@ -0,0 +1,38 @@ +#!/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. +# + +set -o pipefail + +# Go to the Spark project root directory +FWDIR="$(cd `dirname $0`/..; pwd)" +cd $FWDIR + +echo -e "q\n" | sbt/sbt oldDeps/update + +export SPARK_CLASSPATH=`find lib_managed \( -name '*spark*jar' -a -type f \) -printf "%p:" ` +./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore +echo -e "q\n" | sbt/sbt mima-report-binary-issues | grep -v -e "info.*Resolving" +ret_val=$? + +if [ $ret_val != 0 ]; then + echo "NOTE: Exceptions to binary compatibility can be added in project/MimaExcludes.scala" +fi + +rm -f .generated-mima* +exit $ret_val diff --git a/dev/run-tests b/dev/run-tests index 6ad674a2ba127..c82a47ebb618b 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -24,9 +24,6 @@ cd $FWDIR # Remove work directory rm -rf ./work -# Fail fast -set -e -set -o pipefail if test -x "$JAVA_HOME/bin/java"; then declare java_cmd="$JAVA_HOME/bin/java" else @@ -35,6 +32,20 @@ fi JAVA_VERSION=$($java_cmd -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') [ "$JAVA_VERSION" -ge 18 ] && echo "" || echo "[Warn] Java 8 tests will not run because JDK version is < 1.8." +# Partial solution for SPARK-1455. Only run Hive tests if there are sql changes. +if [ -n "$AMPLAB_JENKINS" ]; then + git fetch origin master:master + diffs=`git diff --dirstat master | awk '{ print $2; }' | grep "^sql/"` + if [ -n "$diffs" ]; then + echo "Detected changes in SQL. Will run Hive test suite." + export _RUN_SQL_TESTS=true # exported for PySpark tests + fi +fi + +# Fail fast +set -e +set -o pipefail + echo "=========================================================================" echo "Running Apache RAT checks" echo "=========================================================================" @@ -48,20 +59,23 @@ dev/scalastyle echo "=========================================================================" echo "Running Spark unit tests" echo "=========================================================================" -# echo "q" is needed because sbt on encountering a build file with failure (either resolution or compilation) -# prompts the user for input either q, r, etc to quit or retry. This echo is there to make it not block. -echo -e "q\n" | sbt/sbt assembly test | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" +# echo "q" is needed because sbt on encountering a build file with failure +# (either resolution or compilation) prompts the user for input either q, r, +# etc to quit or retry. This echo is there to make it not block. +if [ -n "$_RUN_SQL_TESTS" ]; then + echo -e "q\n" | SPARK_HIVE=true sbt/sbt clean assembly test | \ + grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" +else + echo -e "q\n" | sbt/sbt clean assembly test | \ + grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" +fi echo "=========================================================================" echo "Running PySpark tests" echo "=========================================================================" -if [ -z "$PYSPARK_PYTHON" ]; then - export PYSPARK_PYTHON=/usr/local/bin/python2.7 -fi ./python/run-tests echo "=========================================================================" echo "Detecting binary incompatibilites with MiMa" echo "=========================================================================" -./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore -echo -e "q\n" | sbt/sbt mima-report-binary-issues | grep -v -e "info.*Resolving" +dev/mima diff --git a/dev/scalastyle b/dev/scalastyle index 19955b9aaaad3..0e8fd5cc8d64c 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -17,7 +17,14 @@ # limitations under the License. # -echo -e "q\n" | sbt/sbt clean scalastyle > scalastyle.txt +echo -e "q\n" | SPARK_HIVE=true sbt/sbt scalastyle > scalastyle.txt +# Check style with YARN alpha built too +echo -e "q\n" | SPARK_HADOOP_VERSION=0.23.9 SPARK_YARN=true sbt/sbt yarn-alpha/scalastyle \ + >> scalastyle.txt +# Check style with YARN built too +echo -e "q\n" | SPARK_HADOOP_VERSION=2.2.0 SPARK_YARN=true sbt/sbt yarn/scalastyle \ + >> scalastyle.txt + ERRORS=$(cat scalastyle.txt | grep -e "\") if test ! -z "$ERRORS"; then echo -e "Scalastyle checks failed at following occurrences:\n$ERRORS" diff --git a/docs/README.md b/docs/README.md index 0678fc5c86706..fd7ba4e0d72ea 100644 --- a/docs/README.md +++ b/docs/README.md @@ -1,22 +1,31 @@ Welcome to the Spark documentation! -This readme will walk you through navigating and building the Spark documentation, which is included here with the Spark source code. You can also find documentation specific to release versions of Spark at http://spark.apache.org/documentation.html. +This readme will walk you through navigating and building the Spark documentation, which is included +here with the Spark source code. You can also find documentation specific to release versions of +Spark at http://spark.apache.org/documentation.html. -Read on to learn more about viewing documentation in plain text (i.e., markdown) or building the documentation yourself. Why build it yourself? So that you have the docs that corresponds to whichever version of Spark you currently have checked out of revision control. +Read on to learn more about viewing documentation in plain text (i.e., markdown) or building the +documentation yourself. Why build it yourself? So that you have the docs that corresponds to +whichever version of Spark you currently have checked out of revision control. ## Generating the Documentation HTML -We include the Spark documentation as part of the source (as opposed to using a hosted wiki, such as the github wiki, as the definitive documentation) to enable the documentation to evolve along with the source code and be captured by revision control (currently git). This way the code automatically includes the version of the documentation that is relevant regardless of which version or release you have checked out or downloaded. +We include the Spark documentation as part of the source (as opposed to using a hosted wiki, such as +the github wiki, as the definitive documentation) to enable the documentation to evolve along with +the source code and be captured by revision control (currently git). This way the code automatically +includes the version of the documentation that is relevant regardless of which version or release +you have checked out or downloaded. -In this directory you will find textfiles formatted using Markdown, with an ".md" suffix. You can read those text files directly if you want. Start with index.md. +In this directory you will find textfiles formatted using Markdown, with an ".md" suffix. You can +read those text files directly if you want. Start with index.md. -The markdown code can be compiled to HTML using the -[Jekyll tool](http://jekyllrb.com). +The markdown code can be compiled to HTML using the [Jekyll tool](http://jekyllrb.com). To use the `jekyll` command, you will need to have Jekyll installed. The easiest way to do this is via a Ruby Gem, see the -[jekyll installation instructions](http://jekyllrb.com/docs/installation). -Compiling the site with Jekyll will create a directory called -_site containing index.html as well as the rest of the compiled files. +[jekyll installation instructions](http://jekyllrb.com/docs/installation). +If not already installed, you need to install `kramdown` with `sudo gem install kramdown`. +Execute `jekyll` from the `docs/` directory. Compiling the site with Jekyll will create a directory +called `_site` containing index.html as well as the rest of the compiled files. You can modify the default Jekyll build as follows: @@ -29,9 +38,11 @@ You can modify the default Jekyll build as follows: ## Pygments -We also use pygments (http://pygments.org) for syntax highlighting in documentation markdown pages, so you will also need to install that (it requires Python) by running `sudo easy_install Pygments`. +We also use pygments (http://pygments.org) for syntax highlighting in documentation markdown pages, +so you will also need to install that (it requires Python) by running `sudo easy_install Pygments`. -To mark a block of code in your markdown to be syntax highlighted by jekyll during the compile phase, use the following sytax: +To mark a block of code in your markdown to be syntax highlighted by jekyll during the compile +phase, use the following sytax: {% highlight scala %} // Your scala code goes here, you can replace scala with many other @@ -42,8 +53,15 @@ To mark a block of code in your markdown to be syntax highlighted by jekyll duri You can build just the Spark scaladoc by running `sbt/sbt doc` from the SPARK_PROJECT_ROOT directory. -Similarly, you can build just the PySpark epydoc by running `epydoc --config epydoc.conf` from the SPARK_PROJECT_ROOT/pyspark directory. +Similarly, you can build just the PySpark epydoc by running `epydoc --config epydoc.conf` from the +SPARK_PROJECT_ROOT/pyspark directory. Documentation is only generated for classes that are listed as +public in `__init__.py`. -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 `sbt/sbt doc` 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 using [epydoc](http://epydoc.sourceforge.net/). +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 `sbt/sbt doc` 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 using [epydoc](http://epydoc.sourceforge.net/). -NOTE: To skip the step of building and copying over the Scala and Python API docs, run `SKIP_API=1 jekyll`. +NOTE: To skip the step of building and copying over the Scala and Python API docs, run `SKIP_API=1 +jekyll`. diff --git a/docs/_config.yml b/docs/_config.yml index d585b8c5ea763..45b78fe724a50 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -7,6 +7,6 @@ SPARK_VERSION: 1.0.0-SNAPSHOT SPARK_VERSION_SHORT: 1.0.0 SCALA_BINARY_VERSION: "2.10" SCALA_VERSION: "2.10.4" -MESOS_VERSION: 0.13.0 -SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net +MESOS_VERSION: 0.18.1 +SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK SPARK_GITHUB_URL: https://github.com/apache/spark diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 5d4dbb7a9c3a9..4ba20e590f2c2 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -9,6 +9,11 @@ {{ page.title }} - Spark {{site.SPARK_VERSION_SHORT}} Documentation + {% if page.redirect %} + + + {% endif %} +
      Stage IdPool NameDescriptionSubmittedDurationTasks: Succeeded/TotalShuffle ReadShuffle Write
      {s.stageId} - - {poolName.get} - - {description}{submissionTime}{formattedDuration} - {makeProgressBar(startedTasks, completedTasks, failedTasks, totalTasks)} + {s.stageId} + + {poolName.get} + {shuffleRead}{shuffleWrite}
      {makeDescription(s)}{submissionTime}{formattedDuration} + {makeProgressBar(startedTasks, completedTasks, failedTasks, totalTasks)} + {shuffleRead}{shuffleWrite}
      Failure Reason{s.failureReason.getOrElse("")}