aboutsummaryrefslogtreecommitdiff
path: root/core
Commit message (Collapse)AuthorAgeFilesLines
* [Spark 1877] ClassNotFoundException when loading RDD with serialized objectsTathagata Das2014-05-191-1/+1
| | | | | | | | | | | | Updated version of #821 Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Ghidireac <bogdang@u448a5b0a73d45358d94a.ant.amazon.com> Closes #835 from tdas/SPARK-1877 and squashes the following commits: f346f71 [Tathagata Das] Addressed Patrick's comments. fee0c5d [Ghidireac] SPARK-1877: ClassNotFoundException when loading RDD with serialized objects
* SPARK-1689: Spark application should die when removed by MasterAaron Davidson2014-05-191-0/+2
| | | | | | | | | | scheduler.error() will mask the error if there are active tasks. Being removed is a cataclysmic event for Spark applications, and should probably be treated as such. Author: Aaron Davidson <aaron@databricks.com> Closes #832 from aarondav/i-love-u and squashes the following commits: 9f1200f [Aaron Davidson] SPARK-1689: Spark application should die when removed by Master
* SPARK-1879. Increase MaxPermSize since some of our builds have many classesMatei Zaharia2014-05-191-1/+3
| | | | | | | | | | | | See https://issues.apache.org/jira/browse/SPARK-1879 -- builds with Hadoop2 and Hive ran out of PermGen space in spark-shell, when those things added up with the Scala compiler. Note that users can still override it by setting their own Java options with this change. Their options will come later in the command string than the -XX:MaxPermSize=128m. Author: Matei Zaharia <matei@databricks.com> Closes #823 from mateiz/spark-1879 and squashes the following commits: 6bc0ee8 [Matei Zaharia] Increase MaxPermSize to 128m since some of our builds have lots of classes
* [SPARK-1876] Windows fixes to deal with latest distribution layout changesMatei Zaharia2014-05-191-1/+1
| | | | | | | | | | | | | | | | | - Look for JARs in the right place - Launch examples the same way as on Unix - Load datanucleus JARs if they exist - Don't attempt to parse local paths as URIs in SparkSubmit, since paths with C:\ are not valid URIs - Also fixed POM exclusion rules for datanucleus (it wasn't properly excluding it, whereas SBT was) Author: Matei Zaharia <matei@databricks.com> Closes #819 from mateiz/win-fixes and squashes the following commits: d558f96 [Matei Zaharia] Fix comment 228577b [Matei Zaharia] Review comments d3b71c7 [Matei Zaharia] Properly exclude datanucleus files in Maven assembly 144af84 [Matei Zaharia] Update Windows scripts to match latest binary package layout
* Make deprecation warning less severePatrick Wendell2014-05-161-6/+6
| | | | | | | | | | Just a small change. I think it's good not to scare people who are using the old options. Author: Patrick Wendell <pwendell@gmail.com> Closes #810 from pwendell/warnings and squashes the following commits: cb8a311 [Patrick Wendell] Make deprecation warning less severe
* [SPARK-1808] Route bin/pyspark through Spark submitAndrew Or2014-05-164-18/+47
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | **Problem.** For `bin/pyspark`, there is currently no other way to specify Spark configuration properties other than through `SPARK_JAVA_OPTS` in `conf/spark-env.sh`. However, this mechanism is supposedly deprecated. Instead, it needs to pick up configurations explicitly specified in `conf/spark-defaults.conf`. **Solution.** Have `bin/pyspark` invoke `bin/spark-submit`, like all of its counterparts in Scala land (i.e. `bin/spark-shell`, `bin/run-example`). This has the additional benefit of making the invocation of all the user facing Spark scripts consistent. **Details.** `bin/pyspark` inherently handles two cases: (1) running python applications and (2) running the python shell. For (1), Spark submit already handles running python applications. For cases in which `bin/pyspark` is given a python file, we can simply call pass the file directly to Spark submit and let it handle the rest. For case (2), `bin/pyspark` starts a python process as before, which launches the JVM as a sub-process. The existing code already provides a code path to do this. All we needed to change is to use `bin/spark-submit` instead of `spark-class` to launch the JVM. This requires modifications to Spark submit to handle the pyspark shell as a special case. This has been tested locally (OSX and Windows 7), on a standalone cluster, and on a YARN cluster. Running IPython also works as before, except now it takes in Spark submit arguments too. Author: Andrew Or <andrewor14@gmail.com> Closes #799 from andrewor14/pyspark-submit and squashes the following commits: bf37e36 [Andrew Or] Minor changes 01066fa [Andrew Or] bin/pyspark for Windows c8cb3bf [Andrew Or] Handle perverse app names (with escaped quotes) 1866f85 [Andrew Or] Windows is not cooperating 456d844 [Andrew Or] Guard against shlex hanging if PYSPARK_SUBMIT_ARGS is not set 7eebda8 [Andrew Or] Merge branch 'master' of github.com:apache/spark into pyspark-submit b7ba0d8 [Andrew Or] Address a few comments (minor) 06eb138 [Andrew Or] Use shlex instead of writing our own parser 05879fa [Andrew Or] Merge branch 'master' of github.com:apache/spark into pyspark-submit a823661 [Andrew Or] Fix --die-on-broken-pipe not propagated properly 6fba412 [Andrew Or] Deal with quotes + address various comments fe4c8a7 [Andrew Or] Update --help for bin/pyspark afe47bf [Andrew Or] Fix spark shell f04aaa4 [Andrew Or] Merge branch 'master' of github.com:apache/spark into pyspark-submit a371d26 [Andrew Or] Route bin/pyspark through Spark submit
* SPARK-1864 Look in spark conf instead of system properties when propagating ↵Michael Armbrust2014-05-161-4/+5
| | | | | | | | | | configuration to executors. Author: Michael Armbrust <michael@databricks.com> Closes #808 from marmbrus/confClasspath and squashes the following commits: 4c31d57 [Michael Armbrust] Look in spark conf instead of system properties when propagating configuration to executors.
* SPARK-1860: Do not cleanup application work/ directories by defaultAaron Davidson2014-05-151-1/+1
| | | | | | | | | | | | This causes an unrecoverable error for applications that are running for longer than 7 days that have jars added to the SparkContext, as the jars are cleaned up even though the application is still running. Author: Aaron Davidson <aaron@databricks.com> Closes #800 from aarondav/shitty-defaults and squashes the following commits: a573fbb [Aaron Davidson] SPARK-1860: Do not cleanup application work/ directories by default
* Typos in SparkHuajian Mao2014-05-152-2/+2
| | | | | | | | | Author: Huajian Mao <huajianmao@gmail.com> Closes #798 from huajianmao/patch-1 and squashes the following commits: 208a454 [Huajian Mao] A typo in Task 1b515af [Huajian Mao] A typo in the message
* Package docsPrashant Sharma2014-05-1424-1/+525
| | | | | | | | | | | | | | This is a few changes based on the original patch by @scrapcodes. Author: Prashant Sharma <prashant.s@imaginea.com> Author: Patrick Wendell <pwendell@gmail.com> Closes #785 from pwendell/package-docs and squashes the following commits: c32b731 [Patrick Wendell] Changes based on Prashant's patch c0463d3 [Prashant Sharma] added eof new line ce8bf73 [Prashant Sharma] Added eof new line to all files. 4c35f2e [Prashant Sharma] SPARK-1563 Add package-info.java and package.scala files for all packages that appear in docs
* Documentation: Encourage use of reduceByKey instead of groupByKey.Patrick Wendell2014-05-142-0/+24
| | | | | | | | | Author: Patrick Wendell <pwendell@gmail.com> Closes #784 from pwendell/group-by-key and squashes the following commits: 9b4505f [Patrick Wendell] Small fix 6347924 [Patrick Wendell] Documentation: Encourage use of reduceByKey instead of groupByKey.
* [SPARK-1840] SparkListenerBus prints out scary error message when terminated ↵Tathagata Das2014-05-141-0/+2
| | | | | | | | | | | | | | | | | | normally Running SparkPi example gave this error. ``` Pi is roughly 3.14374 14/05/14 18:16:19 ERROR Utils: Uncaught exception in thread SparkListenerBus scala.runtime.NonLocalReturnControl$mcV$sp ``` This is due to the catch-all in the SparkListenerBus, which logged control throwable used by scala system Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #783 from tdas/controlexception-fix and squashes the following commits: a466c8d [Tathagata Das] Ignored control exceptions when logging all exceptions.
* [Typo] propertes -> propertiesandrewor142014-05-141-1/+1
| | | | | | | | Author: andrewor14 <andrewor14@gmail.com> Closes #780 from andrewor14/submit-typo and squashes the following commits: e70e057 [andrewor14] propertes -> properties
* String interpolation + some other small changesJacek Laskowski2014-05-141-10/+8
| | | | | | | | | | After having been invited to make the change in https://github.com/apache/spark/commit/6bee01dd04ef73c6b829110ebcdd622d521ea8ff#commitcomment-6284165 by @witgo. Author: Jacek Laskowski <jacek@japila.pl> Closes #748 from jaceklaskowski/sparkenv-string-interpolation and squashes the following commits: be6ebac [Jacek Laskowski] String interpolation + some other small changes
* SPARK-1833 - Have an empty SparkContext constructor.Patrick Wendell2014-05-141-0/+6
| | | | | | | | | | This is nicer than relying on new SparkContext(new SparkConf()) Author: Patrick Wendell <pwendell@gmail.com> Closes #774 from pwendell/spark-context and squashes the following commits: ef9f12f [Patrick Wendell] SPARK-1833 - Have an empty SparkContext constructor.
* SPARK-1829 Sub-second durations shouldn't round to "0 s"Andrew Ash2014-05-141-0/+6
| | | | | | | | | | | | | | | As "99 ms" up to 99 ms As "0.1 s" from 0.1 s up to 0.9 s https://issues.apache.org/jira/browse/SPARK-1829 Compare the first image to the second here: http://imgur.com/RaLEsSZ,7VTlgfo#0 Author: Andrew Ash <andrew@andrewash.com> Closes #768 from ash211/spark-1829 and squashes the following commits: 1c15b8e [Andrew Ash] SPARK-1829 Format sub-second durations more appropriately
* [SPARK-1620] Handle uncaught exceptions in function run by Akka schedulerMark Hamstra2014-05-145-18/+36
| | | | | | | | | | | | | | | | If the intended behavior was that uncaught exceptions thrown in functions being run by the Akka scheduler would end up being handled by the default uncaught exception handler set in Executor, and if that behavior is, in fact, correct, then this is a way to accomplish that. I'm not certain, though, that we shouldn't be doing something different to handle uncaught exceptions from some of these scheduled functions. In any event, this PR covers all of the cases I comment on in [SPARK-1620](https://issues.apache.org/jira/browse/SPARK-1620). Author: Mark Hamstra <markhamstra@gmail.com> Closes #622 from markhamstra/SPARK-1620 and squashes the following commits: 071d193 [Mark Hamstra] refactored post-SPARK-1772 1a6a35e [Mark Hamstra] another style fix d30eb94 [Mark Hamstra] scalastyle 3573ecd [Mark Hamstra] Use wrapped try/catch in Utils.tryOrExit 8fc0439 [Mark Hamstra] Make functions run by the Akka scheduler use Executor's UncaughtExceptionHandler
* [SPARK-1769] Executor loss causes NPE race conditionAndrew Or2014-05-145-26/+35
| | | | | | | | | | | | | | | | | | This PR replaces the Schedulable data structures in Pool.scala with thread-safe ones from java. Note that Scala's `with SynchronizedBuffer` trait is soon to be deprecated in 2.11 because it is ["inherently unreliable"](http://www.scala-lang.org/api/2.11.0/index.html#scala.collection.mutable.SynchronizedBuffer). We should slowly drift away from `SynchronizedBuffer` in other places too. Note that this PR introduces an API-breaking change; `sc.getAllPools` now returns an Array rather than an ArrayBuffer. This is because we want this method to return an immutable copy rather than one may potentially confuse the user if they try to modify the copy, which takes no effect on the original data structure. Author: Andrew Or <andrewor14@gmail.com> Closes #762 from andrewor14/pool-npe and squashes the following commits: 383e739 [Andrew Or] JavaConverters -> JavaConversions 3f32981 [Andrew Or] Merge branch 'master' of github.com:apache/spark into pool-npe 769be19 [Andrew Or] Assorted minor changes 2189247 [Andrew Or] Merge branch 'master' of github.com:apache/spark into pool-npe 05ad9e9 [Andrew Or] Fix test - contains is not the same as containsKey 0921ea0 [Andrew Or] var -> val 07d720c [Andrew Or] Synchronize Schedulable data structures
* SPARK-1801. expose InterruptibleIterator and TaskKilledException in deve...Koert Kuipers2014-05-142-3/+11
| | | | | | | | | | ...loper api Author: Koert Kuipers <koert@tresata.com> Closes #764 from koertkuipers/feat-rdd-developerapi and squashes the following commits: 8516dd2 [Koert Kuipers] SPARK-1801. expose InterruptibleIterator and TaskKilledException in developer api
* Revert "[SPARK-1784] Add a new partitioner to allow specifying # of keys per ↵Patrick Wendell2014-05-132-95/+0
| | | | | | partition" This reverts commit 92cebada09a7e5a00ab48bcb350a9462949c33eb.
* Implement ApproximateCountDistinct for SparkSqllarvaboy2014-05-131-3/+3
| | | | | | | | | | | | | | | | | | | Add the implementation for ApproximateCountDistinct to SparkSql. We use the HyperLogLog algorithm implemented in stream-lib, and do the count in two phases: 1) counting the number of distinct elements in each partitions, and 2) merge the HyperLogLog results from different partitions. A simple serializer and test cases are added as well. Author: larvaboy <larvaboy@gmail.com> Closes #737 from larvaboy/master and squashes the following commits: bd8ef3f [larvaboy] Add support of user-provided standard deviation to ApproxCountDistinct. 9ba8360 [larvaboy] Fix alignment and null handling issues. 95b4067 [larvaboy] Add a test case for count distinct and approximate count distinct. f57917d [larvaboy] Add the parser for the approximate count. a2d5d10 [larvaboy] Add ApproximateCountDistinct aggregates and functions. 7ad273a [larvaboy] Add SparkSql serializer for HyperLogLog. 1d9aacf [larvaboy] Fix a minor typo in the toString method of the Count case class. 653542b [larvaboy] Fix a couple of minor typos.
* [SPARK-1784] Add a new partitioner to allow specifying # of keys per partitionSyed Hashmi2014-05-132-0/+95
| | | | | | | | | | | This change adds a new partitioner which allows users to specify # of keys per partition. Author: Syed Hashmi <shashmi@cloudera.com> Closes #721 from syedhashmi/master and squashes the following commits: 4ca94cc [Syed Hashmi] [SPARK-1784] Add a new partitioner
* [SPARK-1527] change rootDir*.getName to rootDir*.getAbsolutePathYe Xianjin2014-05-131-1/+1
| | | | | | | | | | | | | | | | JIRA issue: [SPARK-1527](https://issues.apache.org/jira/browse/SPARK-1527) getName() only gets the last component of the file path. When deleting test-generated directories, we should pass the generated directory's absolute path to DiskBlockManager. Author: Ye Xianjin <advancedxy@gmail.com> This patch had conflicts when merged, resolved by Committer: Patrick Wendell <pwendell@gmail.com> Closes #436 from advancedxy/SPARK-1527 and squashes the following commits: 4678bab [Ye Xianjin] change rootDir*.getname to rootDir*.getAbsolutePath so the temporary directories are deleted when the test is finished.
* [SPARK-1816] LiveListenerBus dies if a listener throws an exceptionAndrew Or2014-05-134-29/+109
| | | | | | | | | | | | | The solution is to wrap a try / catch / log around the posting of each event to each listener. Author: Andrew Or <andrewor14@gmail.com> Closes #759 from andrewor14/listener-die and squashes the following commits: aee5107 [Andrew Or] Merge branch 'master' of github.com:apache/spark into listener-die 370939f [Andrew Or] Remove two layers of indirection 422d278 [Andrew Or] Explicitly throw an exception instead of 1 / 0 0df0e2a [Andrew Or] Try/catch and log exceptions when posting events
* SPARK-571: forbid return statements in cleaned closuresWilliam Benton2014-05-132-2/+60
| | | | | | | | | | | | | | | This patch checks top-level closure arguments to `ClosureCleaner.clean` for `return` statements and raises an exception if it finds any. This is mainly a user-friendliness addition, since programs with return statements in closure arguments will currently fail upon RDD actions with a less-than-intuitive error message. Author: William Benton <willb@redhat.com> Closes #717 from willb/spark-571 and squashes the following commits: c41eb7d [William Benton] Another test case for SPARK-571 30c42f4 [William Benton] Stylistic cleanups 559b16b [William Benton] Stylistic cleanups from review de13b79 [William Benton] Style fixes 295b6a5 [William Benton] Forbid return statements in closure arguments. b017c47 [William Benton] Added a test for SPARK-571
* SPARK-1815. SparkContext should not be marked DeveloperApiSandy Ryza2014-05-121-2/+0
| | | | | | | | Author: Sandy Ryza <sandy@cloudera.com> Closes #753 from sryza/sandy-spark-1815 and squashes the following commits: 957a8ac [Sandy Ryza] SPARK-1815. SparkContext should not be marked DeveloperApi
* [SPARK-1780] Non-existent SPARK_DAEMON_OPTS is lurking aroundAndrew Or2014-05-121-1/+1
| | | | | | | | | | What they really mean is SPARK_DAEMON_***JAVA***_OPTS Author: Andrew Or <andrewor14@gmail.com> Closes #751 from andrewor14/spark-daemon-opts and squashes the following commits: 70c41f9 [Andrew Or] SPARK_DAEMON_OPTS -> SPARK_DAEMON_JAVA_OPTS
* Typo: resond -> respondAndrew Ash2014-05-121-1/+1
| | | | | | | | Author: Andrew Ash <andrew@andrewash.com> Closes #743 from ash211/patch-4 and squashes the following commits: c959f3b [Andrew Ash] Typo: resond -> respond
* SPARK-1623: Use File objects instead of String's in HTTPBroadcastPatrick Wendell2014-05-121-4/+4
| | | | | | | | | | | | | | This seems strictly better, and I think it's justified only the grounds of clean-up. It might also fix issues with path conversions, but I haven't yet isolated any instance of that happening. /cc @srowen @tdas Author: Patrick Wendell <pwendell@gmail.com> Closes #749 from pwendell/broadcast-cleanup and squashes the following commits: d6d54f2 [Patrick Wendell] SPARK-1623: Use File objects instead of string's in HTTPBroadcast
* Rename testExecutorEnvs --> executorEnvs.Patrick Wendell2014-05-124-9/+8
| | | | | | | | | | | This was changed, but in fact, it's used for things other than tests. So I've changed it back. Author: Patrick Wendell <pwendell@gmail.com> Closes #747 from pwendell/executor-env and squashes the following commits: 36a60a5 [Patrick Wendell] Rename testExecutorEnvs --> executorEnvs.
* SPARK-1798. Tests should clean up temp filesSean Owen2014-05-1214-85/+116
| | | | | | | | | | | | | | | | | | | | Three issues related to temp files that tests generate – these should be touched up for hygiene but are not urgent. Modules have a log4j.properties which directs the unit-test.log output file to a directory like `[module]/target/unit-test.log`. But this ends up creating `[module]/[module]/target/unit-test.log` instead of former. The `work/` directory is not deleted by "mvn clean", in the parent and in modules. Neither is the `checkpoint/` directory created under the various external modules. Many tests create a temp directory, which is not usually deleted. This can be largely resolved by calling `deleteOnExit()` at creation and trying to call `Utils.deleteRecursively` consistently to clean up, sometimes in an `@After` method. _If anyone seconds the motion, I can create a more significant change that introduces a new test trait along the lines of `LocalSparkContext`, which provides management of temp directories for subclasses to take advantage of._ Author: Sean Owen <sowen@cloudera.com> Closes #732 from srowen/SPARK-1798 and squashes the following commits: 5af578e [Sean Owen] Try to consistently delete test temp dirs and files, and set deleteOnExit() for each b21b356 [Sean Owen] Remove work/ and checkpoint/ dirs with mvn clean bdd0f41 [Sean Owen] Remove duplicate module dir in log4j.properties output path for tests
* SPARK-1806: Upgrade Mesos dependency to 0.18.1Bernardo Gomez Palacio2014-05-123-3/+4
| | | | | | | | | | | | | | | | | | | | Enabled Mesos (0.18.1) dependency with shaded protobuf Why is this needed? Avoids any protobuf version collision between Mesos and any other dependency in Spark e.g. Hadoop HDFS 2.2+ or 1.0.4. Ticket: https://issues.apache.org/jira/browse/SPARK-1806 * Should close https://issues.apache.org/jira/browse/SPARK-1433 Author berngp Author: Bernardo Gomez Palacio <bernardo.gomezpalacio@gmail.com> Closes #741 from berngp/feature/SPARK-1806 and squashes the following commits: 5d70646 [Bernardo Gomez Palacio] SPARK-1806: Upgrade Mesos dependency to 0.18.1
* SPARK-1772 Stop catching Throwable, let Executors dieAaron Davidson2014-05-1219-140/+127
| | | | | | | | | | | | | | | | | The main issue this patch fixes is [SPARK-1772](https://issues.apache.org/jira/browse/SPARK-1772), in which Executors may not die when fatal exceptions (e.g., OOM) are thrown. This patch causes Executors to delegate to the ExecutorUncaughtExceptionHandler when a fatal exception is thrown. This patch also continues the fight in the neverending war against `case t: Throwable =>`, by only catching Exceptions in many places, and adding a wrapper for Threads and Runnables to make sure any uncaught exceptions are at least printed to the logs. It also turns out that it is unlikely that the IndestructibleActorSystem actually works, given testing ([here](https://gist.github.com/aarondav/ca1f0cdcd50727f89c0d)). The uncaughtExceptionHandler is not called from the places that we expected it would be. [SPARK-1620](https://issues.apache.org/jira/browse/SPARK-1620) deals with part of this issue, but refactoring our Actor Systems to ensure that exceptions are dealt with properly is a much bigger change, outside the scope of this PR. Author: Aaron Davidson <aaron@databricks.com> Closes #715 from aarondav/throwable and squashes the following commits: f9b9bfe [Aaron Davidson] Remove other redundant 'throw e' e937a0a [Aaron Davidson] Address Prashant and Matei's comments 1867867 [Aaron Davidson] [RFC] SPARK-1772 Stop catching Throwable, let Executors die
* SPARK-1770: Load balance elements when repartitioning.Patrick Wendell2014-05-112-2/+46
| | | | | | | | | | | | | | | | | | | This patch adds better balancing when performing a repartition of an RDD. Previously the elements in the RDD were hash partitioned, meaning if the RDD was skewed certain partitions would end up being very large. This commit adds load balancing of elements across the repartitioned RDD splits. The load balancing is not perfect: a given output partition can have up to N more elements than the average if there are N input partitions. However, some randomization is used to minimize the probabiliy that this happens. Author: Patrick Wendell <pwendell@gmail.com> Closes #727 from pwendell/load-balance and squashes the following commits: f9da752 [Patrick Wendell] Response to Matei's feedback acfa46a [Patrick Wendell] SPARK-1770: Load balance elements when repartitioning.
* remove outdated runtime Information scala homewitgo2014-05-111-2/+1
| | | | | | | | | Author: witgo <witgo@qq.com> Closes #728 from witgo/scala_home and squashes the following commits: cdfd8be [witgo] Merge branch 'master' of https://github.com/apache/spark into scala_home fac094a [witgo] remove outdated runtime Information scala home
* [SPARK-1774] Respect SparkSubmit --jars on YARN (client)Andrew Or2014-05-103-50/+101
| | | | | | | | | | | | | | | | | | | | SparkSubmit ignores `--jars` for YARN client. This is a bug. This PR also automatically adds the application jar to `spark.jar`. Previously, when running as yarn-client, you must specify the jar additionally through `--files` (because `--jars` didn't work). Now you don't have to explicitly specify it through either. Tested on a YARN cluster. Author: Andrew Or <andrewor14@gmail.com> Closes #710 from andrewor14/yarn-jars and squashes the following commits: 35d1928 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-jars c27bf6c [Andrew Or] For yarn-cluster and python, do not add primaryResource to spark.jar c92c5bf [Andrew Or] Minor cleanups 269f9f3 [Andrew Or] Fix format 013d840 [Andrew Or] Fix tests 1407474 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-jars 3bb75e8 [Andrew Or] Allow SparkSubmit --jars to take effect in yarn-client mode
* SPARK-1789. Multiple versions of Netty dependencies cause FlumeStreamSuite ↵Sean Owen2014-05-101-2/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | failure TL;DR is there is a bit of JAR hell trouble with Netty, that can be mostly resolved and will resolve a test failure. I hit the error described at http://apache-spark-user-list.1001560.n3.nabble.com/SparkContext-startup-time-out-td1753.html while running FlumeStreamingSuite, and have for a short while (is it just me?) velvia notes: "I have found a workaround. If you add akka 2.2.4 to your dependencies, then everything works, probably because akka 2.2.4 brings in newer version of Jetty." There are at least 3 versions of Netty in play in the build: - the new Flume 1.4.0 dependency brings in io.netty:netty:3.4.0.Final, and that is the immediate problem - the custom version of akka 2.2.3 depends on io.netty:netty:3.6.6. - but, Spark Core directly uses io.netty:netty-all:4.0.17.Final The POMs try to exclude other versions of netty, but are excluding org.jboss.netty:netty, when in fact older versions of io.netty:netty (not netty-all) are also an issue. The org.jboss.netty:netty excludes are largely unnecessary. I replaced many of them with io.netty:netty exclusions until everything agreed on io.netty:netty-all:4.0.17.Final. But this didn't work, since Akka 2.2.3 doesn't work with Netty 4.x. Down-grading to 3.6.6.Final across the board made some Spark code not compile. If the build *keeps* io.netty:netty:3.6.6.Final as well, everything seems to work. Part of the reason seems to be that Netty 3.x used the old `org.jboss.netty` packages. This is less than ideal, but is no worse than the current situation. So this PR resolves the issue and improves the JAR hell, even if it leaves the existing theoretical Netty 3-vs-4 conflict: - Remove org.jboss.netty excludes where possible, for clarity; they're not needed except with Hadoop artifacts - Add io.netty:netty excludes where needed -- except, let akka keep its io.netty:netty - Change a bit of test code that actually depended on Netty 3.x, to use 4.x equivalent - Update SBT build accordingly A better change would be to update Akka far enough such that it agrees on Netty 4.x, but I don't know if that's feasible. Author: Sean Owen <sowen@cloudera.com> Closes #723 from srowen/SPARK-1789 and squashes the following commits: 43661b7 [Sean Owen] Update and add Netty excludes to prevent some JAR conflicts that cause test issues
* [SPARK-1690] Tolerating empty elements when saving Python RDD to text filesKan Zhang2014-05-101-2/+3
| | | | | | | | | | | Tolerate empty strings in PythonRDD Author: Kan Zhang <kzhang@apache.org> Closes #644 from kanzhang/SPARK-1690 and squashes the following commits: c62ad33 [Kan Zhang] Adding Python doctest 473ec4b [Kan Zhang] [SPARK-1690] Tolerating empty elements when saving Python RDD to text files
* Add Python includes to path before depickling broadcast valuesBouke van der Bijl2014-05-101-5/+5
| | | | | | | | | | | | This fixes https://issues.apache.org/jira/browse/SPARK-1731 by adding the Python includes to the PYTHONPATH before depickling the broadcast values @airhorns Author: Bouke van der Bijl <boukevanderbijl@gmail.com> Closes #656 from bouk/python-includes-before-broadcast and squashes the following commits: 7b0dfe4 [Bouke van der Bijl] Add Python includes to path before depickling broadcast values
* SPARK-1708. Add a ClassTag on Serializer and things that depend on itMatei Zaharia2014-05-1018-42/+65
| | | | | | | | | | | | | | | | | This pull request contains a rebased patch from @heathermiller (https://github.com/heathermiller/spark/pull/1) to add ClassTags on Serializer and types that depend on it (Broadcast and AccumulableCollection). Putting these in the public API signatures now will allow us to use Scala Pickling for serialization down the line without breaking binary compatibility. One question remaining is whether we also want them on Accumulator -- Accumulator is passed as part of a bigger Task or TaskResult object via the closure serializer so it doesn't seem super useful to add the ClassTag there. Broadcast and AccumulableCollection in contrast were being serialized directly. CC @rxin, @pwendell, @heathermiller Author: Matei Zaharia <matei@databricks.com> Closes #700 from mateiz/spark-1708 and squashes the following commits: 1a3d8b0 [Matei Zaharia] Use fake ClassTag in Java 3b449ed [Matei Zaharia] test fix 2209a27 [Matei Zaharia] Code style fixes 9d48830 [Matei Zaharia] Add a ClassTag on Serializer and things that depend on it
* SPARK-1686: keep schedule() calling in the main threadCodingCat2014-05-091-3/+12
| | | | | | | | | | | | | | | | | | | | | | https://issues.apache.org/jira/browse/SPARK-1686 moved from original JIRA (by @markhamstra): In deploy.master.Master, the completeRecovery method is the last thing to be called when a standalone Master is recovering from failure. It is responsible for resetting some state, relaunching drivers, and eventually resuming its scheduling duties. There are currently four places in Master.scala where completeRecovery is called. Three of them are from within the actor's receive method, and aren't problems. The last starts from within receive when the ElectedLeader message is received, but the actual completeRecovery() call is made from the Akka scheduler. That means that it will execute on a different scheduler thread, and Master itself will end up running (i.e., schedule() ) from that Akka scheduler thread. In this PR, I added a new master message TriggerSchedule to trigger the "local" call of schedule() in the scheduler thread Author: CodingCat <zhunansjtu@gmail.com> Closes #639 from CodingCat/SPARK-1686 and squashes the following commits: 81bb4ca [CodingCat] rename variable 69e0a2a [CodingCat] style fix 36a2ac0 [CodingCat] address Aaron's comments ec9b7bb [CodingCat] address the comments 02b37ca [CodingCat] keep schedule() calling in the main thread
* SPARK-1770: Revert accidental(?) fixAaron Davidson2014-05-091-2/+2
| | | | | | | | | | | | | Looks like this change was accidentally committed here: https://github.com/apache/spark/commit/06b15baab25951d124bbe6b64906f4139e037deb but the change does not show up in the PR itself (#704). Other than not intending to go in with that PR, this also broke the test JavaAPISuite.repartition. Author: Aaron Davidson <aaron@databricks.com> Closes #716 from aarondav/shufflerand and squashes the following commits: b1cf70b [Aaron Davidson] SPARK-1770: Revert accidental(?) fix
* Converted bang to ask to avoid scary warning when a block is removedTathagata Das2014-05-081-1/+1
| | | | | | | | | | | | | | | | | | Removing a block through the blockmanager gave a scary warning messages in the driver. ``` 2014-05-08 20:16:19,172 WARN BlockManagerMasterActor: Got unknown message: true 2014-05-08 20:16:19,172 WARN BlockManagerMasterActor: Got unknown message: true 2014-05-08 20:16:19,172 WARN BlockManagerMasterActor: Got unknown message: true ``` This is because the [BlockManagerSlaveActor](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala#L44) would send back an acknowledgement ("true"). But the BlockManagerMasterActor would have sent the RemoveBlock message as a send, not as ask(), so would reject the receiver "true" as a unknown message. @pwendell Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #708 from tdas/bm-fix and squashes the following commits: ed4ef15 [Tathagata Das] Converted bang to ask to avoid scary warning when a block is removed.
* MINOR: Removing dead code.Patrick Wendell2014-05-081-1/+0
| | | | Meant to do this when patching up the last merge.
* SPARK-1775: Unneeded lock in ShuffleMapTask.deserializeInfoSandeep2014-05-081-9/+7
| | | | | | | | | | This was used in the past to have a cache of deserialized ShuffleMapTasks, but that's been removed, so there's no need for a lock. It slows down Spark when task descriptions are large, e.g. due to large lineage graphs or local variables. Author: Sandeep <sandeep@techaddict.me> Closes #707 from techaddict/SPARK-1775 and squashes the following commits: 18d8ebf [Sandeep] SPARK-1775: Unneeded lock in ShuffleMapTask.deserializeInfo This was used in the past to have a cache of deserialized ShuffleMapTasks, but that's been removed, so there's no need for a lock. It slows down Spark when task descriptions are large, e.g. due to large lineage graphs or local variables.
* SPARK-1565 (Addendum): Replace `run-example` with `spark-submit`.Patrick Wendell2014-05-081-2/+2
| | | | | | | | | | | | | Gives a nicely formatted message to the user when `run-example` is run to tell them to use `spark-submit`. Author: Patrick Wendell <pwendell@gmail.com> Closes #704 from pwendell/examples and squashes the following commits: 1996ee8 [Patrick Wendell] Feedback form Andrew 3eb7803 [Patrick Wendell] Suggestions from TD 2474668 [Patrick Wendell] SPARK-1565 (Addendum): Replace `run-example` with `spark-submit`.
* [SPARK-1755] Respect SparkSubmit --name on YARNAndrew Or2014-05-082-8/+11
| | | | | | | | | | | | | | | | | | | | Right now, SparkSubmit ignores the `--name` flag for both yarn-client and yarn-cluster. This is a bug. In client mode, SparkSubmit treats `--name` as a [cluster config](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala#L170) and does not propagate this to SparkContext. In cluster mode, SparkSubmit passes this flag to `org.apache.spark.deploy.yarn.Client`, which only uses it for the [YARN ResourceManager](https://github.com/apache/spark/blob/master/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala#L80), but does not propagate this to SparkContext. This PR ensures that `spark.app.name` is always set if SparkSubmit receives the `--name` flag, which is what the usage promises. This makes it possible for applications to start a SparkContext with an empty conf `val sc = new SparkContext(new SparkConf)`, and inherit the app name from SparkSubmit. Tested both modes on a YARN cluster. Author: Andrew Or <andrewor14@gmail.com> Closes #699 from andrewor14/yarn-app-name and squashes the following commits: 98f6a79 [Andrew Or] Fix tests dea932f [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-app-name c86d9ca [Andrew Or] Respect SparkSubmit --name on YARN
* [SPARK-1745] Move interrupted flag from TaskContext constructor (minor)Andrew Or2014-05-085-22/+17
| | | | | | | | | | | | | | | It makes little sense to start a TaskContext that is interrupted. Indeed, I searched for all use cases of it and didn't find a single instance in which `interrupted` is true on construction. This was inspired by reviewing #640, which adds an additional `@volatile var completed` that is similar. These are not the most urgent changes, but I wanted to push them out before I forget. Author: Andrew Or <andrewor14@gmail.com> Closes #675 from andrewor14/task-context and squashes the following commits: 9575e02 [Andrew Or] Add space 69455d1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into task-context c471490 [Andrew Or] Oops, removed one flag too many. Adding it back. 85311f8 [Andrew Or] Move interrupted flag from TaskContext constructor
* SPARK-1565, update examples to be used with spark-submit script.Prashant Sharma2014-05-081-4/+4
| | | | | | | | | | | | | | | | Commit for initial feedback, basically I am curious if we should prompt user for providing args esp. when its mandatory. And can we skip if they are not ? Also few other things that did not work like `bin/spark-submit examples/target/scala-2.10/spark-examples-1.0.0-SNAPSHOT-hadoop1.0.4.jar --class org.apache.spark.examples.SparkALS --arg 100 500 10 5 2` Not all the args get passed properly, may be I have messed up something will try to sort it out hopefully. Author: Prashant Sharma <prashant.s@imaginea.com> Closes #552 from ScrapCodes/SPARK-1565/update-examples and squashes the following commits: 669dd23 [Prashant Sharma] Review comments 2727e70 [Prashant Sharma] SPARK-1565, update examples to be used with spark-submit script.
* [SPARK-1688] Propagate PySpark worker stderr to driverAndrew Or2014-05-074-101/+123
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | When at least one of the following conditions is true, PySpark cannot be loaded: 1. PYTHONPATH is not set 2. PYTHONPATH does not contain the python directory (or jar, in the case of YARN) 3. The jar does not contain pyspark files (YARN) 4. The jar does not contain py4j files (YARN) However, we currently throw the same random `java.io.EOFException` for all of the above cases, when trying to read from the python daemon's output. This message is super unhelpful. This PR includes the python stderr and the PYTHONPATH in the exception propagated to the driver. Now, the exception message looks something like: ``` Error from python worker: : No module named pyspark PYTHONPATH was: /path/to/spark/python:/path/to/some/jar java.io.EOFException <stack trace> ``` whereas before it was just ``` java.io.EOFException <stack trace> ``` Author: Andrew Or <andrewor14@gmail.com> Closes #603 from andrewor14/pyspark-exception and squashes the following commits: 10d65d3 [Andrew Or] Throwable -> Exception, worker -> daemon 862d1d7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into pyspark-exception a5ed798 [Andrew Or] Use block string and interpolation instead of var (minor) cc09c45 [Andrew Or] Account for the fact that the python daemon may not have terminated yet 444f019 [Andrew Or] Use the new RedirectThread + include system PYTHONPATH aab00ae [Andrew Or] Merge branch 'master' of github.com:apache/spark into pyspark-exception 0cc2402 [Andrew Or] Merge branch 'master' of github.com:apache/spark into pyspark-exception 783efe2 [Andrew Or] Make python daemon stderr indentation consistent 9524172 [Andrew Or] Avoid potential NPE / error stream contention + Move things around 29f9688 [Andrew Or] Add back original exception type e92d36b [Andrew Or] Include python worker stderr in the exception propagated to the driver 7c69360 [Andrew Or] Merge branch 'master' of github.com:apache/spark into pyspark-exception cdbc185 [Andrew Or] Fix python attribute not found exception when PYTHONPATH is not set dcc0353 [Andrew Or] Check both python and system environment variables for PYTHONPATH 6c09c21 [Andrew Or] Validate PYTHONPATH and PySpark modules before starting python workers