Commit graph

5439 commits

Author SHA1 Message Date
Andrew Or aa5002bb96 Defensively allocate memory from global pool
This is an alternative to the existing approach, which evenly distributes the
collective shuffle memory among all running tasks. In the new approach, each
thread requests a chunk of memory whenever its map is about to multiplicatively
grow. If there is sufficient memory in the global pool, the thread allocates it
and grows its map. Otherwise, it spills.

A danger with the previous approach is that a new task may quickly fill up its
map before old tasks finish spilling, potentially causing an OOM. This approach
prevents this scenario as it favors existing tasks over new tasks; any thread
that may step over the boundary of other threads defensively backs off and
starts spilling.

Testing through spark-perf reveals: (1) When no spills have occured, the
performance of external sorting using this memory management approach is
essentially the same as without external sorting. (2) When one or more spills
have occured, the performance of external sorting is a small multiple (3x) worse
2014-01-09 21:43:58 -08:00
Andrew Or d76e1f90a8 Merge github.com:apache/incubator-spark
Conflicts:
	core/src/main/scala/org/apache/spark/SparkEnv.scala
	streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
2014-01-09 21:38:48 -08:00
Patrick Wendell 7b748b83a1 Minor clean-up 2014-01-09 20:42:48 -08:00
Patrick Wendell 300eaa994c Merge pull request #353 from pwendell/ipython-simplify
Simplify and fix pyspark script.

This patch removes compatibility for IPython < 1.0 but fixes the launch
script and makes it much simpler.

I tested this using the three commands in the PySpark documentation page:

1. IPYTHON=1 ./pyspark
2. IPYTHON_OPTS="notebook" ./pyspark
3. IPYTHON_OPTS="notebook --pylab inline" ./pyspark

There are two changes:
- We rely on PYTHONSTARTUP env var to start PySpark
- Removed the quotes around $IPYTHON_OPTS... having quotes
  gloms them together as a single argument passed to `exec` which
  seemed to cause ipython to fail (it instead expects them as
  multiple arguments).
2014-01-09 20:29:51 -08:00
Prabeesh K 645f5e83ee Change clientId to random clientId
Returns a randomly generated client identifier based on the current user's login name and the system time.
2014-01-10 09:33:31 +05:30
Tathagata Das 38d75e18fa Merge remote-tracking branch 'apache/master' into driver-test 2014-01-09 19:31:36 -08:00
Tathagata Das 4a5558ca99 Fixed bugs in reading of checkpoints. 2014-01-10 03:28:39 +00:00
Reynold Xin 4b074fac05 Merge pull request #374 from mateiz/completeness
Add some missing Java API methods

These are primarily for setting job groups, canceling jobs, and setting names on RDDs. Seemed like useful stuff to expose in Java.
2014-01-09 19:03:55 -08:00
Reynold Xin a9d533333d Merge pull request #294 from RongGu/master
Bug fixes for updating the RDD block's memory and disk usage information

Bug fixes for updating the RDD block's memory and disk usage information.
From the code context, we can find that the memSize and diskSize here are both always equal to the size of the block. Actually, they never be zero. Thus, the logic here is wrong for recording the block usage in BlockStatus, especially for the blocks which are dropped from memory to ensure space for the new input rdd blocks. I have tested it that this would cause the storage metrics shown in the Storage webpage wrong and misleading. With this patch, the metrics will be okay.
 Finally, Merry Christmas, guys:)
2014-01-09 18:46:46 -08:00
Patrick Wendell 77ca9e1ba8 Small fix suggested by josh 2014-01-09 18:41:00 -08:00
Patrick Wendell d86a85e9ca Merge pull request #293 from pwendell/standalone-driver
SPARK-998: Support Launching Driver Inside of Standalone Mode

[NOTE: I need to bring the tests up to date with new changes, so for now they will fail]

This patch provides support for launching driver programs inside of a standalone cluster manager. It also supports monitoring and re-launching of driver programs which is useful for long running, recoverable applications such as Spark Streaming jobs. For those jobs, this patch allows a deployment mode which is resilient to the failure of any worker node, failure of a master node (provided a multi-master setup), and even failures of the applicaiton itself, provided they are recoverable on a restart. Driver information, such as the status and logs from a driver, is displayed in the UI

There are a few small TODO's here, but the code is generally feature-complete. They are:
- Bring tests up to date and add test coverage
- Restarting on failure should be optional and maybe off by default.
- See if we can re-use akka connections to facilitate clients behind a firewall

A sensible place to start for review would be to look at the `DriverClient` class which presents users the ability to launch their driver program. I've also added an example program (`DriverSubmissionTest`) that allows you to test this locally and play around with killing workers, etc. Most of the code is devoted to persisting driver state in the cluster manger, exposing it in the UI, and dealing correctly with various types of failures.

Instructions to test locally:
- `sbt/sbt assembly/assembly examples/assembly`
- start a local version of the standalone cluster manager

```
./spark-class org.apache.spark.deploy.client.DriverClient \
  -j -Dspark.test.property=something \
  -e SPARK_TEST_KEY=SOMEVALUE \
  launch spark://10.99.1.14:7077 \
  ../path-to-examples-assembly-jar \
  org.apache.spark.examples.DriverSubmissionTest 1000 some extra options --some-option-here -X 13
```
- Go in the UI and make sure it started correctly, look at the output etc
- Kill workers, the driver program, masters, etc.
2014-01-09 18:37:52 -08:00
Matei Zaharia c43eb00644 Fix bug added when we changed AppDescription.maxCores to an Option
The Scala compiler warned about this -- we were comparing an Option
against an integer now.
2014-01-09 18:14:20 -08:00
Matei Zaharia 142921c6c0 Add some missing Java API methods 2014-01-09 18:11:12 -08:00
Patrick Wendell 26cdb5f68a Merge pull request #372 from pwendell/log4j-fix-1
Send logs to stderr by default (instead of stdout).
2014-01-09 17:16:34 -08:00
jerryshao dd7fd0c443 Upgrade Kafka dependecy to 0.8.0 release version 2014-01-10 08:56:11 +08:00
Patrick Wendell 2af98198ad Send logs to stderr by default (instead of stdout). 2014-01-09 15:57:44 -08:00
Matei Zaharia 12f414ed43 Merge pull request #362 from mateiz/conf-getters
Use typed getters for configuration settings

This improves some of the code style after SPARK-544.
2014-01-09 15:31:30 -08:00
Tathagata Das f1d206c6b4 Merge branch 'standalone-driver' into driver-test
Conflicts:
	core/src/main/scala/org/apache/spark/SparkContext.scala
	core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
	examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java
	streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
	streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
	streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
	streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
2014-01-09 15:06:24 -08:00
Thomas Graves 9bdfbc0492 Update based on review comments 2014-01-09 15:45:33 -06:00
Tathagata Das 6f713e2a3e Changed the way StreamingContext finds and reads checkpoint files, and added JavaStreamingContext.getOrCreate. 2014-01-09 13:42:04 -08:00
Patrick Wendell 67b9a33628 Some usability improvements 2014-01-09 12:42:37 -08:00
Patrick Wendell 35f80da21a Set default logging to WARN for Spark streaming examples.
This programatically sets the log level to WARN by default for streaming
tests. If the user has already specified a log4j.properties file,
the user's file will take precedence over this default.
2014-01-09 10:42:58 -08:00
Thomas Graves 5fccd7051d add application type 2014-01-09 10:32:35 -06:00
Thomas Graves c617083e47 yarn-client addJar fix and misc other 2014-01-09 10:24:35 -06:00
Pillis 181471906e SPARK-961 Add a Vector.random() method 2014-01-09 10:16:19 +01:00
Reynold Xin 365cac9465 Merge pull request #361 from rxin/clean
Minor style cleanup. Mostly on indenting & line width changes.

Focused on the few important files since they are the files that new contributors usually read first.
2014-01-09 00:56:16 -08:00
Reynold Xin 73c724e123 Merge pull request #368 from pwendell/sbt-fix
Don't delegate to users `sbt`.

This changes our `sbt/sbt` script to not delegate to the user's `sbt`
even if it is present. If users already have sbt installed and they
want to use their own sbt, we'd expect them to just call sbt directly
from within Spark. We no longer set any enironment variables or anything
from this script, so they should just launch sbt directly on their own.

There are a number of hard-to-debug issues which can come from the
current appraoch. One is if the user is unaware of an existing sbt
installation and now without explanation their build breaks because
they haven't configured options correctly (such as permgen size)
within their sbt (reported by @patmcdonough). Another is if the user has a much older version
of sbt hanging around, in which case some of the older versions
don't acutally work well when newer verisons of sbt are specified
in the build file (reported by @marmbrus). A third is if the user
has done some other modification to their sbt script, such as
setting it to delegate to sbt/sbt in Spark, and this causes
that to break (also reported by @marmbrus).

So to keep things simple let's just avoid this path and
remove it. Any user who already has sbt and wants to build
spark with it should be able to understand easily how to do it.
2014-01-09 00:32:19 -08:00
Reynold Xin 295d82583a Minor update on SparkContext.broadcast's JavaDoc. 2014-01-09 00:30:22 -08:00
Patrick Wendell 49cbf48bcc Small typo fix 2014-01-09 00:12:34 -08:00
Matei Zaharia a01f3401e3 Use typed getters for configuration settings 2014-01-09 00:07:29 -08:00
Patrick Wendell 4d2e388e6a Don't delegate to users sbt.
This changes our `sbt/sbt` script to not delegate to the user's `sbt`
even if it is present. If users already have sbt installed and they
want to use their own sbt, we'd expect them to just call sbt directly
from within Spark. We no longer set any enironment variables or anything
from this script, so they should just launch sbt directly on their own.

There are a number of hard-to-debug issues which can come from the
current appraoch. One is if the user is unaware of an existing sbt
installation and now without explanation their build breaks because
they haven't configured options correctly (such as permgen size)
within their sbt. Another is if the user has a much older version
of sbt hanging around, in which case some of the older versions
don't acutally work well when newer verisons of sbt are specified
in the build file (reported by @marmbrus). A third is if the user
has done some other modification to their sbt script, such as
setting it to delegate to sbt/sbt in Spark, and this causes
that to break (also reported by @marmbrus).

So to keep things simple let's just avoid this path and
remove it. Any user who already has sbt and wants to build
spark with it should be able to understand easily how to do it.
2014-01-08 23:56:53 -08:00
Patrick Wendell dceedb4660 Merge pull request #364 from pwendell/fix
Fixing config option "retained_stages" => "retainedStages".

This is a very esoteric option and it's out of sync with the style we use.
So it seems fitting to fix it for 0.9.0.
2014-01-08 23:19:28 -08:00
Prashant Sharma 59b03e015d Fixes corresponding to Reynolds feedback comments 2014-01-09 12:26:30 +05:30
Patrick Wendell 112c0a1776 Fixing config option "retained_stages" => "retainedStages".
This is a very esoteric option and it's out of sync with the style we use.
So it seems fitting to fix it for 0.9.0.
2014-01-08 21:16:16 -08:00
Patrick Wendell 0f9d2ace6b Adding polling to driver submission client. 2014-01-08 16:56:26 -08:00
Reynold Xin 46f6a3b6aa Minor style cleanup. Mostly on indenting & line width changes. 2014-01-08 14:55:04 -08:00
Reynold Xin 04d83fc37f Merge pull request #360 from witgo/master
fix make-distribution.sh show version: command not found
2014-01-08 11:55:37 -08:00
Reynold Xin 56ebfeaa52 Merge pull request #357 from hsaputra/set_boolean_paramname
Set boolean param name for call to SparkHadoopMapReduceUtil.newTaskAttemptID

Set boolean param name for call to SparkHadoopMapReduceUtil.newTaskAttemptID to make it clear which param being set.
2014-01-08 11:50:06 -08:00
Patrick Wendell bdeaeafbda Merge pull request #358 from pwendell/add-cdh
Add CDH Repository to Maven Build

At some point this was removed from the Maven build... so I'm adding it back. It's needed for the Hadoop2 tests we run on Jenkins and it's also included in the SBT build.
2014-01-08 11:48:39 -08:00
Reynold Xin 5cae05f59e Merge pull request #356 from hsaputra/remove_deprecated_cleanup_method
Remove calls to deprecated mapred's OutputCommitter.cleanupJob

Since Hadoop 1.0.4 the mapred OutputCommitter.commitJob should do cleanup job via call to OutputCommitter.cleanupJob,

Remove SparkHadoopWriter.cleanup since it is used only by PairRDDFunctions.

In fact the implementation of mapred OutputCommitter.commitJob looks like this:

  public void commitJob(JobContext jobContext) throws IOException {
    cleanupJob(jobContext);
  }
2014-01-08 11:47:28 -08:00
walker d942f95d7e Merge remote branch 'upstream/master' 2014-01-09 01:22:26 +08:00
liguoqiang cf4aaf92d6 fix make-distribution.sh show version: command not found 2014-01-09 00:34:53 +08:00
Thomas Graves 6eef78d769 Merge pull request #345 from colorant/yarn
support distributing extra files to worker for yarn client mode

So that user doesn't need to package all dependency into one assemble jar as spark app jar
2014-01-08 08:49:20 -06:00
Tathagata Das a17cc602ac More bug fixes. 2014-01-08 04:12:05 -08:00
Tathagata Das 0b7a132d03 Modified checkpoing file clearing policy. 2014-01-08 03:22:06 -08:00
Prashant Sharma 277b4a36c5 we clone hadoop key and values by default and reuse if specified. 2014-01-08 16:32:55 +05:30
Patrick Wendell 3209a86f39 Add CDH Repository to Maven Build 2014-01-08 01:21:17 -08:00
Patrick Wendell 62b08faac5 Adding mockito to maven build 2014-01-08 00:45:41 -08:00
Patrick Wendell bc81ce040d Merge remote-tracking branch 'apache-github/master' into standalone-driver
Conflicts:
	core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
	pom.xml
2014-01-08 00:38:31 -08:00
Henry Saputra aa56585d21 Resolve PR review over 100 chars 2014-01-08 00:38:29 -08:00