Commit graph

2060 commits

Author SHA1 Message Date
Ali Ghodsi 05a0df2b9e Makes Spark SIMR ready. 2013-10-24 11:59:51 -07:00
Josh Rosen 768eb9c962 Remove redundant Java Function call() definitions
This should fix SPARK-902, an issue where some
Java API Function classes could cause
AbstractMethodErrors when user code is compiled
using the Eclipse compiler.

Thanks to @MartinWeindel for diagnosing this
problem.

(This PR subsumes / closes #30)
2013-10-22 14:26:52 -07:00
Patrick Wendell ab5ece19a3 Formatting cleanup 2013-10-22 13:03:08 -07:00
Patrick Wendell c22046b3cc Minor clean-up in review 2013-10-22 11:00:50 -07:00
Patrick Wendell 7de0ea4d42 Response to code review and adding some more tests 2013-10-22 11:00:50 -07:00
Patrick Wendell 2fa3c4c49c Fix for Spark-870.
This patch fixes a bug where the Spark UI didn't display the correct number of total
tasks if the number of tasks in a Stage doesn't equal the number of RDD partitions.

It also cleans up the listener API a bit by embedding this information in the
StageInfo class rather than passing it seperately.
2013-10-22 11:00:25 -07:00
Patrick Wendell a854f5bfcf SPARK-940: Do not directly pass Stage objects to SparkListener. 2013-10-22 11:00:06 -07:00
Matei Zaharia a0e08f0fb9 Merge pull request #82 from JoshRosen/map-output-tracker-refactoring
Split MapOutputTracker into Master/Worker classes

Previously, MapOutputTracker contained fields and methods that were only applicable to the master or worker instances.  This commit introduces a MasterMapOutputTracker class to prevent the master-specific methods from being accessed on workers.

I also renamed a few methods and made others protected/private.
2013-10-22 10:20:43 -07:00
Aaron Davidson 053ef949ac Merge ShufflePerfTester patch into shuffle block consolidation 2013-10-21 22:17:53 -07:00
Reynold Xin a51359c917 Merge pull request #95 from aarondav/perftest
Minor: Put StoragePerfTester in org/apache/
2013-10-21 20:33:29 -07:00
Aaron Davidson 97053c4a91 Put StoragePerfTester in org/apache/ 2013-10-21 20:25:40 -07:00
Aaron Davidson 0071f0899c Fix mesos urls
This was a bug I introduced in https://github.com/apache/incubator-spark/pull/71
Previously, we explicitly removed the mesos:// part; with PR 71, this no longer occured.
2013-10-21 15:56:14 -07:00
Aaron Davidson 4aa0ba1df7 Remove executorId from Task.run() 2013-10-21 12:19:15 -07:00
Patrick Wendell aa61bfd399 Merge pull request #88 from rxin/clean
Made the following traits/interfaces/classes non-public:

Made the following traits/interfaces/classes non-public:
SparkHadoopWriter
SparkHadoopMapRedUtil
SparkHadoopMapReduceUtil
SparkHadoopUtil
PythonAccumulatorParam
BlockManagerSlaveActor
2013-10-21 11:57:05 -07:00
Aaron Davidson 444162afe7 Documentation update 2013-10-20 22:59:45 -07:00
Aaron Davidson 947fceaa73 Close shuffle writers during failure & remove executorId from TaskContext 2013-10-20 22:47:10 -07:00
Patrick Wendell 35886f3474 Merge pull request #41 from pwendell/shuffle-benchmark
Provide Instrumentation for Shuffle Write Performance

Shuffle write performance can have a major impact on the performance of jobs. This patch adds a few pieces of instrumentation related to shuffle writes. They are:

1. A listing of the time spent performing blocking writes for each task. This is implemented by keeping track of the aggregate delay seen by many individual writes.
2. An undocumented option `spark.shuffle.sync` which forces shuffle data to sync to disk. This is necessary for measuring shuffle performance in the absence of the OS buffer cache.
3. An internal utility which micro-benchmarks write throughput for simulated shuffle outputs.

I'm going to do some performance testing on this to see whether these small timing calls add overhead. From a feature perspective, however, I consider this complete. Any feedback is appreciated.
2013-10-20 22:20:32 -07:00
Reynold Xin 5b9380e017 Merge pull request #89 from rxin/executor
Don't setup the uncaught exception handler in local mode.

This avoids unit test failures for Spark streaming.

    java.util.concurrent.RejectedExecutionException: Task org.apache.spark.streaming.JobManager$JobHandler@38cf728d rejected from java.util.concurrent.ThreadPoolExecutor@3b69a41e[Terminated, pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 14]
	at java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(ThreadPoolExecutor.java:2048)
	at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:821)
	at java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:1372)
	at org.apache.spark.streaming.JobManager.runJob(JobManager.scala:54)
	at org.apache.spark.streaming.Scheduler$$anonfun$generateJobs$2.apply(Scheduler.scala:108)
	at org.apache.spark.streaming.Scheduler$$anonfun$generateJobs$2.apply(Scheduler.scala:108)
	at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:60)
	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
	at org.apache.spark.streaming.Scheduler.generateJobs(Scheduler.scala:108)
	at org.apache.spark.streaming.Scheduler$$anonfun$1.apply$mcVJ$sp(Scheduler.scala:41)
	at org.apache.spark.streaming.util.RecurringTimer.org$apache$spark$streaming$util$RecurringTimer$$loop(RecurringTimer.scala:66)
	at org.apache.spark.streaming.util.RecurringTimer$$anon$1.run(RecurringTimer.scala:34)
2013-10-20 21:03:51 -07:00
Reynold Xin b4d8478454 Made JobLogger public again and some minor cleanup. 2013-10-20 18:59:28 -07:00
Aaron Davidson 4b68ddf3d0 Cleanup old shuffle file metadata from memory 2013-10-20 17:56:41 -07:00
Matei Zaharia edc5e3f8f4 Merge pull request #75 from JoshRosen/block-manager-cleanup
Code de-duplication in BlockManager

The BlockManager has a few methods that duplicate most of their code.  This pull request extracts the duplicated code into private doPut(), doGetLocal(), and doGetRemote() methods that unify the storing/reading of bytes or objects.

I believe that I preserved the logic of the original code, but I'd appreciate some help in reviewing this.
2013-10-20 17:18:06 -07:00
Aaron Davidson 42a049723d Address Josh and Reynold's comments 2013-10-20 16:11:59 -07:00
Josh Rosen 1fa5baf9ab Unwrap a long line that actually fits. 2013-10-20 14:50:21 -07:00
Josh Rosen 640f253a65 Fix test failures in local mode due to updateEpoch 2013-10-20 14:49:05 -07:00
Josh Rosen 68d6806ea4 Minor cleanup based on @aarondav's code review. 2013-10-20 13:20:14 -07:00
Reynold Xin 7414805e4e Don't setup the uncaught exception handler in local mode.
This avoids unit test failures for Spark streaming.
2013-10-20 13:03:48 -07:00
Reynold Xin 8e1937f8ba Made the following traits/interfaces/classes non-public:
SparkHadoopWriter
SparkHadoopMapRedUtil
SparkHadoopMapReduceUtil
SparkHadoopUtil
PythonAccumulatorParam
JobLogger
BlockManagerSlaveActor
2013-10-20 12:22:07 -07:00
Reynold Xin 2a7ae1736a Merge pull request #84 from rxin/kill1
Added documentation for setJobGroup. Also some minor cleanup in SparkContext.
2013-10-20 11:45:21 -07:00
Aaron Davidson 38b8048f29 Fix compiler errors
Whoops. Last-second changes require testing too, it seems.
2013-10-20 11:03:36 -07:00
Reynold Xin fabd05dabc Updated setGroupId documentation and marked dagSchedulerSource and blockManagerSource as private in SparkContext. 2013-10-20 10:54:30 -07:00
Matei Zaharia e4abb75d70 Merge pull request #85 from rxin/clean
Moved the top level spark package object from spark to org.apache.spark

This is a pretty annoying documentation bug ...
2013-10-20 09:38:37 -07:00
Aaron Davidson 136b9b3a3e Basic shuffle file consolidation
The Spark shuffle phase can produce a large number of files, as one file is created
per mapper per reducer. For large or repeated jobs, this often produces millions of
shuffle files, which sees extremely degredaded performance from the OS file system.
This patch seeks to reduce that burden by combining multipe shuffle files into one.

This PR draws upon the work of Jason Dai in https://github.com/mesos/spark/pull/669.
However, it simplifies the design in order to get the majority of the gain with less
overall intellectual and code burden. The vast majority of code in this pull request
is a refactor to allow the insertion of a clean layer of indirection between logical
block ids and physical files. This, I feel, provides some design clarity in addition
to enabling shuffle file consolidation.

The main goal is to produce one shuffle file per reducer per active mapper thread.
This allows us to isolate the mappers (simplifying the failure modes), while still
allowing us to reduce the number of mappers tremendously for large tasks. In order
to accomplish this, we simply create a new set of shuffle files for every parallel
task, and return the files to a pool which will be given out to the next run task.
2013-10-20 02:58:26 -07:00
Aaron Davidson 861dc409d7 Refactor of DiskStore for shuffle file consolidation
The main goal of this refactor was to allow the interposition of a new layer which
maps logical BlockIds to physical locations other than a file with the same name
as the BlockId. In particular, BlockIds will need to be mappable to chunks of files,
as multiple will be stored in the same file.

In order to accomplish this, the following changes have been made:
- Creation of DiskBlockManager, which manages the association of logical BlockIds
  to physical disk locations (called FileSegments). By default, Blocks are simply
  mapped to physical files of the same name, as before.
- The DiskStore now indirects all requests for a given BlockId through the DiskBlockManager
  in order to resolve the actual File location.
- DiskBlockObjectWriter has been merged into BlockObjectWriter.
- The Netty PathResolver has been changed to map BlockIds into FileSegments, as this
  codepath is the only one that uses Netty, and that is likely to remain the case.

Overall, I think this refactor produces a clearer division between the logical Block
paradigm and their physical on-disk location. There is now an explicit (and documented)
mapping from one to the other.
2013-10-20 02:48:41 -07:00
Reynold Xin 8396a6649e Moved the top level spark package object from spark to org.apache.spark 2013-10-19 23:26:15 -07:00
Reynold Xin eb9bf69462 Added documentation for setJobGroup. Also some minor cleanup in SparkContext. 2013-10-19 23:16:44 -07:00
Josh Rosen 9159d2d09d Split MapOutputTracker into Master/Worker classes.
Previously, MapOutputTracker contained fields and methods that
were only applicable to the master or worker instances.  This
commit introduces a MasterMapOutputTracker class to prevent
the master-specific methods from being accessed on workers.

I also renamed a few methods and made others protected/private.
2013-10-19 20:01:22 -07:00
Josh Rosen 867d8fdf2a De-duplicate code in dropOld[Non]BroadcastBlocks. 2013-10-19 19:53:12 -07:00
Josh Rosen 6925a1322b Code de-duplication in put() and putBytes(). 2013-10-19 19:53:12 -07:00
Josh Rosen 8279185651 De-duplication in getRemote() and getRemoteBytes(). 2013-10-19 19:53:12 -07:00
Josh Rosen babccb695e De-duplication in getLocal() and getLocalBytes(). 2013-10-19 19:52:10 -07:00
Reynold Xin 6511bbe2ad Merge pull request #78 from mosharaf/master
Removed BitTorrentBroadcast and TreeBroadcast.

TorrentBroadcast replaces both.
2013-10-19 11:34:56 -07:00
Mosharaf Chowdhury 29617c27a1 Removed BitTorrentBroadcast and TreeBroadcast. TorrentBroadcast is replacing both. 2013-10-18 23:54:11 -07:00
Matei Zaharia 599dcb0ddf Merge pull request #74 from rxin/kill
Job cancellation via job group id.

This PR adds a simple API to group together a set of jobs belonging to a thread and threads spawned from it. It also allows the cancellation of all jobs in this group.

An example:

    sc.setJobDescription("this_is_the_group_id", "some job description")
    sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.count()

In a separate thread:

    sc.cancelJobGroup("this_is_the_group_id")
2013-10-18 22:49:00 -07:00
Reynold Xin 806f3a3adb Job cancellation via job group id. 2013-10-18 21:46:08 -07:00
Matei Zaharia e5316d0685 Merge pull request #68 from mosharaf/master
Faster and stable/reliable broadcast

HttpBroadcast is noticeably slow, but the alternatives (TreeBroadcast or BitTorrentBroadcast) are notoriously unreliable. The main problem with them is they try to manage the memory for the pieces of a broadcast themselves. Right now, the BroadcastManager does not know which machines the tasks reading from a broadcast variable is running and when they have finished. Consequently, we try to guess and often guess wrong, which blows up the memory usage and kills/hangs jobs.

This very simple implementation solves the problem by not trying to manage the intermediate pieces; instead, it offloads that duty to the BlockManager which is quite good at juggling blocks. Otherwise, it is very similar to the BitTorrentBroadcast implementation (without fancy optimizations). And it runs much faster than HttpBroadcast we have right now.

I've been using this for another project for last couple of weeks, and just today did some benchmarking against the Http one. The following shows the improvements for increasing broadcast size for cold runs. Each line represent the number of receivers.
![fix-bc-first](https://f.cloud.github.com/assets/232966/1349342/ffa149e4-36e7-11e3-9fa6-c74555829356.png)

After the first broadcast is over, i.e., after JVM is wormed up and for HttpBroadcast the server is already running (I think), the following are the improvements for warm runs.
![fix-bc-succ](https://f.cloud.github.com/assets/232966/1349352/5a948bae-36e8-11e3-98ce-34f19ebd33e0.jpg)
The curves are not as nice as the cold runs, but the improvements are obvious, specially for larger broadcasts and more receivers.

Depending on how it goes, we should deprecate and/or remove old TreeBroadcast and BitTorrentBroadcast implementations, and hopefully, SPARK-889 will not be necessary any more.
2013-10-18 20:30:56 -07:00
Aaron Davidson 74737264c4 Spark shell exits if it cannot create SparkContext
Mainly, this occurs if you provide a messed up MASTER url (one that doesn't match one
of our regexes). Previously, we would default to Mesos, fail, and then start the shell
anyway, except that any Spark command would fail.
2013-10-17 18:51:19 -07:00
Mosharaf Chowdhury 6a84e40efe Merge remote-tracking branch 'upstream/master' 2013-10-17 13:14:33 -07:00
Mosharaf Chowdhury 35b2415fb3 Code styling. Updated doc. 2013-10-17 13:14:12 -07:00
Mosharaf Chowdhury e663750488 Removed unused code.
Changes to match Spark coding style.
2013-10-17 00:19:50 -07:00
Mosharaf Chowdhury a8d0981832 Fixes for the new BlockId naming convention. 2013-10-16 21:33:33 -07:00