Commit graph

1091 commits

Author SHA1 Message Date
Matei Zaharia e966284409 SPARK-2045 Sort-based shuffle
This adds a new ShuffleManager based on sorting, as described in https://issues.apache.org/jira/browse/SPARK-2045. The bulk of the code is in an ExternalSorter class that is similar to ExternalAppendOnlyMap, but sorts key-value pairs by partition ID and can be used to create a single sorted file with a map task's output. (Longer-term I think this can take on the remaining functionality in ExternalAppendOnlyMap and replace it so we don't have code duplication.)

The main TODOs still left are:
- [x] enabling ExternalSorter to merge across spilled files
  - [x] with an Ordering
  - [x] without an Ordering, using the keys' hash codes
- [x] adding more tests (e.g. a version of our shuffle suite that runs on this)
- [x] rebasing on top of the size-tracking refactoring in #1165 when that is merged
- [x] disabling spilling if spark.shuffle.spill is set to false

Despite this though, this seems to work pretty well (running successfully in cases where the hash shuffle would OOM, such as 1000 reduce tasks on executors with only 1G memory), and it seems to be comparable in speed or faster than hash-based shuffle (it will create much fewer files for the OS to keep track of). So I'm posting it to get some early feedback.

After these TODOs are done, I'd also like to enable ExternalSorter to sort data within each partition by a key as well, which will allow us to use it to implement external spilling in reduce tasks in `sortByKey`.

Author: Matei Zaharia <matei@databricks.com>

Closes #1499 from mateiz/sort-based-shuffle and squashes the following commits:

bd841f9 [Matei Zaharia] Various review comments
d1c137fd [Matei Zaharia] Various review comments
a611159 [Matei Zaharia] Compile fixes due to rebase
62c56c8 [Matei Zaharia] Fix ShuffledRDD sometimes not returning Tuple2s.
f617432 [Matei Zaharia] Fix a failing test (seems to be due to change in SizeTracker logic)
9464d5f [Matei Zaharia] Simplify code and fix conflicts after latest rebase
0174149 [Matei Zaharia] Add cleanup behavior and cleanup tests for sort-based shuffle
eb4ee0d [Matei Zaharia] Remove customizable element type in ShuffledRDD
fa2e8db [Matei Zaharia] Allow nextBatchStream to be called after we're done looking at all streams
a34b352 [Matei Zaharia] Fix tracking of indices within a partition in SpillReader, and add test
03e1006 [Matei Zaharia] Add a SortShuffleSuite that runs ShuffleSuite with sort-based shuffle
3c7ff1f [Matei Zaharia] Obey the spark.shuffle.spill setting in ExternalSorter
ad65fbd [Matei Zaharia] Rebase on top of Aaron's Sorter change, and use Sorter in our buffer
44d2a93 [Matei Zaharia] Use estimateSize instead of atGrowThreshold to test collection sizes
5686f71 [Matei Zaharia] Optimize merging phase for in-memory only data:
5461cbb [Matei Zaharia] Review comments and more tests (e.g. tests with 1 element per partition)
e9ad356 [Matei Zaharia] Update ContextCleanerSuite to make sure shuffle cleanup tests use hash shuffle (since they were written for it)
c72362a [Matei Zaharia] Added bug fix and test for when iterators are empty
de1fb40 [Matei Zaharia] Make trait SizeTrackingCollection private[spark]
4988d16 [Matei Zaharia] tweak
c1b7572 [Matei Zaharia] Small optimization
ba7db7f [Matei Zaharia] Handle null keys in hash-based comparator, and add tests for collisions
ef4e397 [Matei Zaharia] Support for partial aggregation even without an Ordering
4b7a5ce [Matei Zaharia] More tests, and ability to sort data if a total ordering is given
e1f84be [Matei Zaharia] Fix disk block manager test
5a40a1c [Matei Zaharia] More tests
614f1b4 [Matei Zaharia] Add spill metrics to map tasks
cc52caf [Matei Zaharia] Add more error handling and tests for error cases
bbf359d [Matei Zaharia] More work
3a56341 [Matei Zaharia] More partial work towards sort-based shuffle
7a0895d [Matei Zaharia] Some more partial work towards sort-based shuffle
b615476 [Matei Zaharia] Scaffolding for sort-based shuffle
2014-07-30 18:07:59 -07:00
Reynold Xin 774142f555 [SPARK-2521] Broadcast RDD object (instead of sending it along with every task)
This is a resubmission of #1452. It was reverted because it broke the build.

Currently (as of Spark 1.0.1), Spark sends RDD object (which contains closures) using Akka along with the task itself to the executors. This is inefficient because all tasks in the same stage use the same RDD object, but we have to send RDD object multiple times to the executors. This is especially bad when a closure references some variable that is very large. The current design led to users having to explicitly broadcast large variables.

The patch uses broadcast to send RDD objects and the closures to executors, and use Akka to only send a reference to the broadcast RDD/closure along with the partition specific information for the task. For those of you who know more about the internals, Spark already relies on broadcast to send the Hadoop JobConf every time it uses the Hadoop input, because the JobConf is large.

The user-facing impact of the change include:

1. Users won't need to decide what to broadcast anymore, unless they would want to use a large object multiple times in different operations
2. Task size will get smaller, resulting in faster scheduling and higher task dispatch throughput.

In addition, the change will simplify some internals of Spark, eliminating the need to maintain task caches and the complex logic to broadcast JobConf (which also led to a deadlock recently).

A simple way to test this:
```scala
val a = new Array[Byte](1000*1000); scala.util.Random.nextBytes(a);
sc.parallelize(1 to 1000, 1000).map { x => a; x }.groupBy { x => a; x }.count
```

Numbers on 3 r3.8xlarge instances on EC2
```
master branch: 5.648436068 s, 4.715361895 s, 5.360161877 s
with this change: 3.416348793 s, 1.477846558 s, 1.553432156 s
```

Author: Reynold Xin <rxin@apache.org>

Closes #1498 from rxin/broadcast-task and squashes the following commits:

f7364db [Reynold Xin] Code review feedback.
f8535dc [Reynold Xin] Fixed the style violation.
252238d [Reynold Xin] Serialize the final task closure as well as ShuffleDependency in taskBinary.
111007d [Reynold Xin] Fix broadcast tests.
797c247 [Reynold Xin] Properly send SparkListenerStageSubmitted and SparkListenerStageCompleted.
bab1d8b [Reynold Xin] Check for NotSerializableException in submitMissingTasks.
cf38450 [Reynold Xin] Use TorrentBroadcastFactory.
991c002 [Reynold Xin] Use HttpBroadcast.
de779f8 [Reynold Xin] Fix TaskContextSuite.
cc152fc [Reynold Xin] Don't cache the RDD broadcast variable.
d256b45 [Reynold Xin] Fixed unit test failures. One more to go.
cae0af3 [Reynold Xin] [SPARK-2521] Broadcast RDD object (instead of sending it along with every task).
2014-07-30 09:27:43 -07:00
Koert Kuipers 7c5fc28af4 SPARK-2543: Allow user to set maximum Kryo buffer size
Author: Koert Kuipers <koert@tresata.com>

Closes #735 from koertkuipers/feat-kryo-max-buffersize and squashes the following commits:

15f6d81 [Koert Kuipers] change default for spark.kryoserializer.buffer.max.mb to 64mb and add some documentation
1bcc22c [Koert Kuipers] Merge branch 'master' into feat-kryo-max-buffersize
0c9f8eb [Koert Kuipers] make default for kryo max buffer size 16MB
143ec4d [Koert Kuipers] test resizable buffer in kryo Output
0732445 [Koert Kuipers] support setting maxCapacity to something different than capacity in kryo Output
2014-07-30 00:26:14 -07:00
Andrew Or 4ce92ccaf7 [SPARK-2260] Fix standalone-cluster mode, which was broken
The main thing was that spark configs were not propagated to the driver, and so applications that do not specify `master` or `appName` automatically failed. This PR fixes that and a couple of miscellaneous things that are related.

One thing that may or may not be an issue is that the jars must be available on the driver node. In `standalone-cluster` mode, this effectively means these jars must be available on all the worker machines, since the driver is launched on one of them. The semantics here are not the same as `yarn-cluster` mode,  where all the relevant jars are uploaded to a distributed cache automatically and shipped to the containers. This is probably not a concern, but still worth a mention.

Author: Andrew Or <andrewor14@gmail.com>

Closes #1538 from andrewor14/standalone-cluster and squashes the following commits:

8c11a0d [Andrew Or] Clean up imports / comments (minor)
2678d13 [Andrew Or] Handle extraJavaOpts properly
7660547 [Andrew Or] Merge branch 'master' of github.com:apache/spark into standalone-cluster
6f64a9b [Andrew Or] Revert changes in YARN
2f2908b [Andrew Or] Fix tests
ed01491 [Andrew Or] Don't go overboard with escaping
8e105e1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into standalone-cluster
b890949 [Andrew Or] Abstract usages of converting spark opts to java opts
79f63a3 [Andrew Or] Move sparkProps into javaOpts
78752f8 [Andrew Or] Fix tests
5a9c6c7 [Andrew Or] Fix line too long
c141a00 [Andrew Or] Don't display "unknown app" on driver log pages
d7e2728 [Andrew Or] Avoid deprecation warning in standalone Client
6ceb14f [Andrew Or] Allow relevant configs to propagate to standalone Driver
7f854bc [Andrew Or] Fix test
855256e [Andrew Or] Fix standalone-cluster mode
fd9da51 [Andrew Or] Formatting changes (minor)
2014-07-29 23:52:09 -07:00
Xiangrui Meng 2e6efcacea [SPARK-2568] RangePartitioner should run only one job if data is balanced
As of Spark 1.0, RangePartitioner goes through data twice: once to compute the count and once to do sampling. As a result, to do sortByKey, Spark goes through data 3 times (once to count, once to sample, and once to sort).

`RangePartitioner` should go through data only once, collecting samples from input partitions as well as counting. If the data is balanced, this should give us a good sketch. If we see big partitions, we re-sample from them in order to collect enough items.

The downside is that we need to collect more from each partition in the first pass. An alternative solution is caching the intermediate result and decide whether to fetch the data after.

Author: Xiangrui Meng <meng@databricks.com>
Author: Reynold Xin <rxin@apache.org>

Closes #1562 from mengxr/range-partitioner and squashes the following commits:

6cc2551 [Xiangrui Meng] change foreach to for
eb39b08 [Xiangrui Meng] Merge branch 'master' into range-partitioner
eb95dd8 [Xiangrui Meng] separate sketching and determining bounds impl
c436d30 [Xiangrui Meng] fix binary metrics unit tests
db58a55 [Xiangrui Meng] add unit tests
a6e35d6 [Xiangrui Meng] minor update
60be09e [Xiangrui Meng] remove importance sampler
9ee9992 [Xiangrui Meng] update range partitioner to run only one job on roughly balanced data
cc12f47 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into range-part
06ac2ec [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into range-part
17bcbf3 [Reynold Xin] Added seed.
badf20d [Reynold Xin] Renamed the method.
6940010 [Reynold Xin] Reservoir sampling implementation.
2014-07-29 22:16:20 -07:00
Doris Xin dc9653641f [SPARK-2082] stratified sampling in PairRDDFunctions that guarantees exact sample size
Implemented stratified sampling that guarantees exact sample size using ScaRSR with two passes over the RDD for sampling without replacement and three passes for sampling with replacement.

Author: Doris Xin <doris.s.xin@gmail.com>
Author: Xiangrui Meng <meng@databricks.com>

Closes #1025 from dorx/stratified and squashes the following commits:

245439e [Doris Xin] moved minSamplingRate to getUpperBound
eaf5771 [Doris Xin] bug fixes.
17a381b [Doris Xin] fixed a merge issue and a failed unit
ea7d27f [Doris Xin] merge master
b223529 [Xiangrui Meng] use approx bounds for poisson fix poisson mean for waitlisting add unit tests for Java
b3013a4 [Xiangrui Meng] move math3 back to test scope
eecee5f [Doris Xin] Merge branch 'master' into stratified
f4c21f3 [Doris Xin] Reviewer comments
a10e68d [Doris Xin] style fix
a2bf756 [Doris Xin] Merge branch 'master' into stratified
680b677 [Doris Xin] use mapPartitionWithIndex instead
9884a9f [Doris Xin] style fix
bbfb8c9 [Doris Xin] Merge branch 'master' into stratified
ee9d260 [Doris Xin] addressed reviewer comments
6b5b10b [Doris Xin] Merge branch 'master' into stratified
254e03c [Doris Xin] minor fixes and Java API.
4ad516b [Doris Xin] remove unused imports from PairRDDFunctions
bd9dc6e [Doris Xin] unit bug and style violation fixed
1fe1cff [Doris Xin] Changed fractionByKey to a map to enable arg check
944a10c [Doris Xin] [SPARK-2145] Add lower bound on sampling rate
0214a76 [Doris Xin] cleanUp
90d94c0 [Doris Xin] merge master
9e74ab5 [Doris Xin] Separated out most of the logic in sampleByKey
7327611 [Doris Xin] merge master
50581fc [Doris Xin] added a TODO for logging in python
46f6c8c [Doris Xin] fixed the NPE caused by closures being cleaned before being passed into the aggregate function
7e1a481 [Doris Xin] changed the permission on SamplingUtil
1d413ce [Doris Xin] fixed checkstyle issues
9ee94ee [Doris Xin] [SPARK-2082] stratified sampling in PairRDDFunctions that guarantees exact sample size
e3fd6a6 [Doris Xin] Merge branch 'master' into takeSample
7cab53a [Doris Xin] fixed import bug in rdd.py
ffea61a [Doris Xin] SPARK-1939: Refactor takeSample method in RDD
1441977 [Doris Xin] SPARK-1939 Refactor takeSample method in RDD to use ScaSRS
2014-07-29 12:49:44 -07:00
Andrew Or ecf30ee7e7 [SPARK-1777] Prevent OOMs from single partitions
**Problem.** When caching, we currently unroll the entire RDD partition before making sure we have enough free memory. This is a common cause for OOMs especially when (1) the BlockManager has little free space left in memory, and (2) the partition is large.

**Solution.** We maintain a global memory pool of `M` bytes shared across all threads, similar to the way we currently manage memory for shuffle aggregation. Then, while we unroll each partition, periodically check if there is enough space to continue. If not, drop enough RDD blocks to ensure we have at least `M` bytes to work with, then try again. If we still don't have enough space to unroll the partition, give up and drop the block to disk directly if applicable.

**New configurations.**
- `spark.storage.bufferFraction` - the value of `M` as a fraction of the storage memory. (default: 0.2)
- `spark.storage.safetyFraction` - a margin of safety in case size estimation is slightly off. This is the equivalent of the existing `spark.shuffle.safetyFraction`. (default 0.9)

For more detail, see the [design document](https://issues.apache.org/jira/secure/attachment/12651793/spark-1777-design-doc.pdf). Tests pending for performance and memory usage patterns.

Author: Andrew Or <andrewor14@gmail.com>

Closes #1165 from andrewor14/them-rdd-memories and squashes the following commits:

e77f451 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
c7c8832 [Andrew Or] Simplify logic + update a few comments
269d07b [Andrew Or] Very minor changes to tests
6645a8a [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
b7e165c [Andrew Or] Add new tests for unrolling blocks
f12916d [Andrew Or] Slightly clean up tests
71672a7 [Andrew Or] Update unrollSafely tests
369ad07 [Andrew Or] Correct ensureFreeSpace and requestMemory behavior
f4d035c [Andrew Or] Allow one thread to unroll multiple blocks
a66fbd2 [Andrew Or] Rename a few things + update comments
68730b3 [Andrew Or] Fix weird scalatest behavior
e40c60d [Andrew Or] Fix MIMA excludes
ff77aa1 [Andrew Or] Fix tests
1a43c06 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
b9a6eee [Andrew Or] Simplify locking behavior on unrollMemoryMap
ed6cda4 [Andrew Or] Formatting fix (super minor)
f9ff82e [Andrew Or] putValues -> putIterator + putArray
beb368f [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
8448c9b [Andrew Or] Fix tests
a49ba4d [Andrew Or] Do not expose unroll memory check period
69bc0a5 [Andrew Or] Always synchronize on putLock before unrollMemoryMap
3f5a083 [Andrew Or] Simplify signature of ensureFreeSpace
dce55c8 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
8288228 [Andrew Or] Synchronize put and unroll properly
4f18a3d [Andrew Or] bufferFraction -> unrollFraction
28edfa3 [Andrew Or] Update a few comments / log messages
728323b [Andrew Or] Do not synchronize every 1000 elements
5ab2329 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
129c441 [Andrew Or] Fix bug: Use toArray rather than array
9a65245 [Andrew Or] Update a few comments + minor control flow changes
57f8d85 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
abeae4f [Andrew Or] Add comment clarifying the MEMORY_AND_DISK case
3dd96aa [Andrew Or] AppendOnlyBuffer -> Vector (+ a few small changes)
f920531 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
0871835 [Andrew Or] Add an effective storage level interface to BlockManager
64e7d4c [Andrew Or] Add/modify a few comments (minor)
8af2f35 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
4f4834e [Andrew Or] Use original storage level for blocks dropped to disk
ecc8c2d [Andrew Or] Fix binary incompatibility
24185ea [Andrew Or] Avoid dropping a block back to disk if reading from disk
2b7ee66 [Andrew Or] Fix bug in SizeTracking*
9b9a273 [Andrew Or] Fix tests
20eb3e5 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
649bdb3 [Andrew Or] Document spark.storage.bufferFraction
a10b0e7 [Andrew Or] Add initial memory request threshold + rename a few things
e9c3cb0 [Andrew Or] cacheMemoryMap -> unrollMemoryMap
198e374 [Andrew Or] Unfold -> unroll
0d50155 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
d9d02a8 [Andrew Or] Remove unused param in unfoldSafely
ec728d8 [Andrew Or] Add tests for safe unfolding of blocks
22b2209 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
078eb83 [Andrew Or] Add check for hasNext in PrimitiveVector.iterator
0871535 [Andrew Or] Fix tests in BlockManagerSuite
d68f31e [Andrew Or] Safely unfold blocks for all memory puts
5961f50 [Andrew Or] Fix tests
195abd7 [Andrew Or] Refactor: move unfold logic to MemoryStore
1e82d00 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
3ce413e [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
d5dd3b4 [Andrew Or] Free buffer memory in finally
ea02eec [Andrew Or] Fix tests
b8e1d9c [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
a8704c1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
e1b8b25 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
87aa75c [Andrew Or] Fix mima excludes again (typo)
11eb921 [Andrew Or] Clarify comment (minor)
50cae44 [Andrew Or] Remove now duplicate mima exclude
7de5ef9 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
df47265 [Andrew Or] Fix binary incompatibility
6d05a81 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
f94f5af [Andrew Or] Update a few comments (minor)
776aec9 [Andrew Or] Prevent OOM if a single RDD partition is too large
bbd3eea [Andrew Or] Fix CacheManagerSuite to use Array
97ea499 [Andrew Or] Change BlockManager interface to use Arrays
c12f093 [Andrew Or] Add SizeTrackingAppendOnlyBuffer and tests
2014-07-27 16:08:16 -07:00
Matei Zaharia 985705301e SPARK-2684: Update ExternalAppendOnlyMap to take an iterator as input
This will decrease object allocation from the "update" closure used in map.changeValue.

Author: Matei Zaharia <matei@databricks.com>

Closes #1607 from mateiz/spark-2684 and squashes the following commits:

b7d89e6 [Matei Zaharia] Add insertAll for Iterables too, and fix some code style
561fc97 [Matei Zaharia] Update ExternalAppendOnlyMap to take an iterator as input
2014-07-27 11:20:20 -07:00
bpaulin c183b92c3c [SPARK-2279] Added emptyRDD method to Java API
Added emptyRDD method to Java API with tests.

Author: bpaulin <bob@bobpaulin.com>

Closes #1597 from bobpaulin/SPARK-2279 and squashes the following commits:

5ad57c2 [bpaulin] [SPARK-2279] Added emptyRDD method to Java API
2014-07-26 10:27:09 -07:00
Reynold Xin 9d8666cac8 Part of [SPARK-2456] Removed some HashMaps from DAGScheduler by storing information in Stage.
This is part of the scheduler cleanup/refactoring effort to make the scheduler code easier to maintain.

@kayousterhout @markhamstra please take a look ...

Author: Reynold Xin <rxin@apache.org>

Closes #1561 from rxin/dagSchedulerHashMaps and squashes the following commits:

1c44e15 [Reynold Xin] Clear pending tasks in submitMissingTasks.
620a0d1 [Reynold Xin] Use filterKeys.
5b54404 [Reynold Xin] Code review feedback.
c1e9a1c [Reynold Xin] Removed some HashMaps from DAGScheduler by storing information in Stage.
2014-07-25 18:45:02 -07:00
Kay Ousterhout 37ad3b7245 [SPARK-1726] [SPARK-2567] Eliminate zombie stages in UI.
Due to problems with when we update runningStages (in DAGScheduler.scala)
and how we decide to send a SparkListenerStageCompleted message to
SparkListeners, sometimes stages can be shown as "running" in the UI forever
(even after they have failed).  This issue can manifest when stages are
resubmitted with 0 tasks, or when the DAGScheduler catches non-serializable
tasks. The problem also resulted in a (small) memory leak in the DAGScheduler,
where stages can stay in runningStages forever. This commit fixes
that problem and adds a unit test.

Thanks tsudukim for helping to look into this issue!

cc markhamstra rxin

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #1566 from kayousterhout/dag_fix and squashes the following commits:

217d74b [Kay Ousterhout] [SPARK-1726] [SPARK-2567] Eliminate zombie stages in UI.
2014-07-25 15:14:13 -07:00
Yin Huai 32bcf9af94 [SPARK-2683] unidoc failed because org.apache.spark.util.CallSite uses Java keywords as value names
Renaming `short` to `shortForm` and `long` to `longForm`.

JIRA: https://issues.apache.org/jira/browse/SPARK-2683

Author: Yin Huai <huai@cse.ohio-state.edu>

Closes #1585 from yhuai/SPARK-2683 and squashes the following commits:

5ddb843 [Yin Huai] "short" and "long" are Java keyworks. In order to generate javadoc, renaming "short" to "shortForm" and "long" to "longForm".
2014-07-25 11:14:51 -07:00
Matei Zaharia 8529ced35c SPARK-2657 Use more compact data structures than ArrayBuffer in groupBy & cogroup
JIRA: https://issues.apache.org/jira/browse/SPARK-2657

Our current code uses ArrayBuffers for each group of values in groupBy, as well as for the key's elements in CoGroupedRDD. ArrayBuffers have a lot of overhead if there are few values in them, which is likely to happen in cases such as join. In particular, they have a pointer to an Object[] of size 16 by default, which is 24 bytes for the array header + 128 for the pointers in there, plus at least 32 for the ArrayBuffer data structure. This patch replaces the per-group buffers with a CompactBuffer class that can store up to 2 elements more efficiently (in fields of itself) and acts like an ArrayBuffer beyond that. For a key's elements in CoGroupedRDD, we use an Array of CompactBuffers instead of an ArrayBuffer of ArrayBuffers.

There are some changes throughout the code to deal with CoGroupedRDD returning Array instead. We can also decide not to do that but CoGroupedRDD is a `DeveloperAPI` so I think it's okay to change it here.

Author: Matei Zaharia <matei@databricks.com>

Closes #1555 from mateiz/compact-groupby and squashes the following commits:

845a356 [Matei Zaharia] Lower initial size of CompactBuffer's vector to 8
07621a7 [Matei Zaharia] Review comments
0c1cd12 [Matei Zaharia] Don't use varargs in CompactBuffer.apply
bdc8a39 [Matei Zaharia] Small tweak to +=, and typos
f61f040 [Matei Zaharia] Fix line lengths
59da88b0 [Matei Zaharia] Fix line lengths
197cde8 [Matei Zaharia] Make CompactBuffer extend Seq to make its toSeq more efficient
775110f [Matei Zaharia] Change CoGroupedRDD to give (K, Array[Iterable[_]]) to avoid wrappers
9b4c6e8 [Matei Zaharia] Use CompactBuffer in CoGroupedRDD
ed577ab [Matei Zaharia] Use CompactBuffer in groupByKey
10f0de1 [Matei Zaharia] A CompactBuffer that's more memory-efficient than ArrayBuffer for small buffers
2014-07-25 00:32:32 -07:00
Sandy Ryza e34922a221 SPARK-2310. Support arbitrary Spark properties on the command line with ...
...spark-submit

The PR allows invocations like
  spark-submit --class org.MyClass --spark.shuffle.spill false myjar.jar

Author: Sandy Ryza <sandy@cloudera.com>

Closes #1253 from sryza/sandy-spark-2310 and squashes the following commits:

1dc9855 [Sandy Ryza] More doc and cleanup
00edfb9 [Sandy Ryza] Review comments
91b244a [Sandy Ryza] Change format to --conf PROP=VALUE
8fabe77 [Sandy Ryza] SPARK-2310. Support arbitrary Spark properties on the command line with spark-submit
2014-07-23 23:11:26 -07:00
Rui Li 91903e0a50 SPARK-2277: clear host->rack info properly
Hi mridulm, I just think of this issue of [#1212](https://github.com/apache/spark/pull/1212): I added FakeRackUtil to hold the host -> rack mapping. It should be cleaned up after use so that it won't mess up with test cases others may add later.
Really sorry about this.

Author: Rui Li <rui.li@intel.com>

Closes #1454 from lirui-intel/SPARK-2277-fix-UT and squashes the following commits:

f8ea25c [Rui Li] SPARK-2277: clear host->rack info properly
2014-07-23 16:23:24 -07:00
Xiangrui Meng 4c7243e109 [SPARK-2617] Correct doc and usages of preservesPartitioning
The name `preservesPartitioning` is ambiguous: 1) preserves the indices of partitions, 2) preserves the partitioner. The latter is correct and `preservesPartitioning` should really be called `preservesPartitioner` to avoid confusion. Unfortunately, this is already part of the API and we cannot change. We should be clear in the doc and fix wrong usages.

This PR

1. adds notes in `maPartitions*`,
2. makes `RDD.sample` preserve partitioner,
3. changes `preservesPartitioning` to false in  `RDD.zip` because the keys of the first RDD are no longer the keys of the zipped RDD,
4. fixes some wrong usages in MLlib.

Author: Xiangrui Meng <meng@databricks.com>

Closes #1526 from mengxr/preserve-partitioner and squashes the following commits:

b361e65 [Xiangrui Meng] update doc based on pwendell's comments
3b1ba19 [Xiangrui Meng] update doc
357575c [Xiangrui Meng] fix unit test
20b4816 [Xiangrui Meng] Merge branch 'master' into preserve-partitioner
d1caa65 [Xiangrui Meng] add doc to explain preservesPartitioning fix wrong usage of preservesPartitioning make sample preserse partitioning
2014-07-23 00:58:55 -07:00
Aaron Davidson 85d3596e65 SPARK-2047: Introduce an in-mem Sorter, and use it to reduce mem usage
### Why and what?
Currently, the AppendOnlyMap performs an "in-place" sort by converting its array of [key, value, key, value] pairs into a an array of [(key, value), (key, value)] pairs. However, this causes us to allocate many Tuple2 objects, which come at a nontrivial overhead.

This patch adds a Sorter API, intended for in memory sorts, which simply ports the Android Timsort implementation (available under Apache v2) and abstracts the interface in a way which introduces no more than 1 virtual function invocation of overhead at each abstraction point.

Please compare our port of the Android Timsort sort with the original implementation: http://www.diffchecker.com/wiwrykcl

### Memory implications
An AppendOnlyMap contains N kv pairs, which results in roughly 2N elements within its underlying array. Each of these elements is 4 bytes wide in a [compressed OOPS](https://wikis.oracle.com/display/HotSpotInternals/CompressedOops) system, which is the default.

Today's approach immediately allocates N Tuple2 objects, which take up 24N bytes in total (exposed via YourKit), and undergoes a Java sort. The Java 6 version immediately copies the entire array (4N bytes here), while the Java 7 version has a worst-case allocation of half the array (2N bytes).
This results in a worst-case sorting overhead of 24N + 2N = 26N bytes (for Java 7).

The Sorter does not require allocating any tuples, but since it uses Timsort, it may copy up to half the entire array in the worst case.
This results in a worst-case sorting overhead of 4N bytes.

Thus, we have reduced the worst-case overhead of the sort by roughly 22 bytes times the number of elements.

### Performance implications
As the destructiveSortedIterator is used for spilling in an ExternalAppendOnlyMap, the purpose of this patch is to provide stability by reducing memory usage rather than improve performance. However, because it implements Timsort, it also brings a substantial performance boost over our prior implementation.

Here are the results of a microbenchmark that sorted 25 million, randomly distributed (Float, Int) pairs. The Java Arrays.sort() tests were run **only on the keys**, and thus moved less data. Our current implementation is called "Tuple-sort using Arrays.sort()" while the new implementation is "KV-array using Sorter".

<table>
<tr><th>Test</th><th>First run (JDK6)</th><th>Average of 10 (JDK6)</th><th>First run (JDK7)</th><th>Average of 10 (JDK7)</th></tr>
<tr><td>primitive Arrays.sort()</td><td>3216 ms</td><td>1190 ms</td><td>2724 ms</td><td>131 ms (!!)</td></tr>
<tr><td>Arrays.sort()</td><td>18564 ms</td><td>2006 ms</td><td>13201 ms</td><td>878 ms</td></tr>
<tr><td>Tuple-sort using Arrays.sort()</td><td>31813 ms</td><td>3550 ms</td><td>20990 ms</td><td>1919 ms</td></tr>
<tr><td><b>KV-array using Sorter</b></td><td></td><td></td><td><b>15020 ms</b></td><td><b>834 ms</b></td></tr>
</table>

The results show that this Sorter performs exactly as expected (after the first run) -- it is as fast as the Java 7 Arrays.sort() (which shares the same algorithm), but is significantly faster than the Tuple-sort on Java 6 or 7.

In short, this patch should significantly improve performance for users running either Java 6 or 7.

Author: Aaron Davidson <aaron@databricks.com>

Closes #1502 from aarondav/sort and squashes the following commits:

652d936 [Aaron Davidson] Update license, move Sorter to java src
a7b5b1c [Aaron Davidson] fix licenses
5c0efaf [Aaron Davidson] Update tmpLength
ec395c8 [Aaron Davidson] Ignore benchmark (again) and fix docs
034bf10 [Aaron Davidson] Change to Apache v2 Timsort
b97296c [Aaron Davidson] Don't try to run benchmark on Jenkins + private[spark]
6307338 [Aaron Davidson] SPARK-2047: Introduce an in-mem Sorter, and use it to reduce mem usage
2014-07-22 11:58:53 -07:00
Sandy Ryza 9564f85489 SPARK-2564. ShuffleReadMetrics.totalBlocksRead is redundant
Author: Sandy Ryza <sandy@cloudera.com>

Closes #1474 from sryza/sandy-spark-2564 and squashes the following commits:

35b8388 [Sandy Ryza] Fix compile error on upmerge
7b985fb [Sandy Ryza] Fix test compile error
43f79e6 [Sandy Ryza] SPARK-2564. ShuffleReadMetrics.totalBlocksRead is redundant
2014-07-20 14:45:34 -07:00
Reynold Xin fa51b0fb5b [SPARK-2598] RangePartitioner's binary search does not use the given Ordering
We should fix this in branch-1.0 as well.

Author: Reynold Xin <rxin@apache.org>

Closes #1500 from rxin/rangePartitioner and squashes the following commits:

c0a94f5 [Reynold Xin] [SPARK-2598] RangePartitioner's binary search does not use the given Ordering.
2014-07-20 11:06:06 -07:00
Reynold Xin 1efb3698b6 Revert "[SPARK-2521] Broadcast RDD object (instead of sending it along with every task)."
This reverts commit 7b8cd17525.
2014-07-19 16:56:22 -07:00
Reynold Xin 7b8cd17525 [SPARK-2521] Broadcast RDD object (instead of sending it along with every task).
Currently (as of Spark 1.0.1), Spark sends RDD object (which contains closures) using Akka along with the task itself to the executors. This is inefficient because all tasks in the same stage use the same RDD object, but we have to send RDD object multiple times to the executors. This is especially bad when a closure references some variable that is very large. The current design led to users having to explicitly broadcast large variables.

The patch uses broadcast to send RDD objects and the closures to executors, and use Akka to only send a reference to the broadcast RDD/closure along with the partition specific information for the task. For those of you who know more about the internals, Spark already relies on broadcast to send the Hadoop JobConf every time it uses the Hadoop input, because the JobConf is large.

The user-facing impact of the change include:

1. Users won't need to decide what to broadcast anymore, unless they would want to use a large object multiple times in different operations
2. Task size will get smaller, resulting in faster scheduling and higher task dispatch throughput.

In addition, the change will simplify some internals of Spark, eliminating the need to maintain task caches and the complex logic to broadcast JobConf (which also led to a deadlock recently).

A simple way to test this:
```scala
val a = new Array[Byte](1000*1000); scala.util.Random.nextBytes(a);
sc.parallelize(1 to 1000, 1000).map { x => a; x }.groupBy { x => a; x }.count
```

Numbers on 3 r3.8xlarge instances on EC2
```
master branch: 5.648436068 s, 4.715361895 s, 5.360161877 s
with this change: 3.416348793 s, 1.477846558 s, 1.553432156 s
```

Author: Reynold Xin <rxin@apache.org>

Closes #1452 from rxin/broadcast-task and squashes the following commits:

762e0be [Reynold Xin] Warn large broadcasts.
ade6eac [Reynold Xin] Log broadcast size.
c3b6f11 [Reynold Xin] Added a unit test for clean up.
754085f [Reynold Xin] Explain why broadcasting serialized copy of the task.
04b17f0 [Reynold Xin] [SPARK-2521] Broadcast RDD object once per TaskSet (instead of sending it for every task).
2014-07-18 23:52:47 -07:00
Kay Ousterhout 7b971b91ca [SPARK-2571] Correctly report shuffle read metrics.
Currently, shuffle read metrics are incorrectly reported when stages have multiple shuffle dependencies (they are set to be the metrics from just one of the shuffle dependencies, rather than the accumulated metrics from all of the shuffle dependencies).  This fixes that problem, and should probably be back-ported to the 0.9 branch.

Thanks ryanra for discovering this problem!

cc rxin andrewor14

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #1476 from kayousterhout/join_bug and squashes the following commits:

0203a16 [Kay Ousterhout] Fix broken unit tests.
f463c2e [Kay Ousterhout] [SPARK-2571] Correctly report shuffle read metrics.
2014-07-18 14:40:32 -07:00
Reynold Xin 586e716e47 Reservoir sampling implementation.
This is going to be used in https://issues.apache.org/jira/browse/SPARK-2568

Author: Reynold Xin <rxin@apache.org>

Closes #1478 from rxin/reservoirSample and squashes the following commits:

17bcbf3 [Reynold Xin] Added seed.
badf20d [Reynold Xin] Renamed the method.
6940010 [Reynold Xin] Reservoir sampling implementation.
2014-07-18 12:41:50 -07:00
Reynold Xin 72e9021eaf [SPARK-2299] Consolidate various stageIdTo* hash maps in JobProgressListener
This should reduce memory usage for the web ui as well as slightly increase its speed in draining the UI event queue.

@andrewor14

Author: Reynold Xin <rxin@apache.org>

Closes #1262 from rxin/ui-consolidate-hashtables and squashes the following commits:

1ac3f97 [Reynold Xin] Oops. Properly handle description.
f5736ad [Reynold Xin] Code review comments.
b8828dc [Reynold Xin] Merge branch 'master' into ui-consolidate-hashtables
7a7b6c4 [Reynold Xin] Revert css change.
f959bb8 [Reynold Xin] [SPARK-2299] Consolidate various stageIdTo* hash maps in JobProgressListener to speed it up.
63256f5 [Reynold Xin] [SPARK-2320] Reduce <pre> block font size.
2014-07-17 18:58:48 -07:00
Aaron Davidson 7c23c0dc3e [SPARK-2412] CoalescedRDD throws exception with certain pref locs
If the first pass of CoalescedRDD does not find the target number of locations AND the second pass finds new locations, an exception is thrown, as "groupHash.get(nxt_replica).get" is not valid.

The fix is just to add an ArrayBuffer to groupHash for that replica if it didn't already exist.

Author: Aaron Davidson <aaron@databricks.com>

Closes #1337 from aarondav/2412 and squashes the following commits:

f587b5d [Aaron Davidson] getOrElseUpdate
3ad8a3c [Aaron Davidson] [SPARK-2412] CoalescedRDD throws exception with certain pref locs
2014-07-17 01:01:14 -07:00
Reynold Xin 7c8d123225 [SPARK-2317] Improve task logging.
We use TID to indicate task logging. However, TID itself does not capture stage or retries, making it harder to correlate with the application itself. This pull request changes all logging messages for tasks to include both the TID and the stage id, stage attempt, task id, and task attempt.  I've consulted various people but unfortunately this is a really hard task.

Driver log looks like:

```
14/06/28 18:53:29 INFO DAGScheduler: Submitting 10 missing tasks from Stage 0 (MappedRDD[1] at map at <console>:13)
14/06/28 18:53:29 INFO TaskSchedulerImpl: Adding task set 0.0 with 10 tasks
14/06/28 18:53:29 INFO TaskSetManager: Re-computing pending task lists.
14/07/15 19:44:40 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 0, localhost, PROCESS_LOCAL, 1855 bytes)
14/07/15 19:44:40 INFO TaskSetManager: Starting task 1.0 in stage 1.0 (TID 1, localhost, PROCESS_LOCAL, 1855 bytes)
14/07/15 19:44:40 INFO TaskSetManager: Starting task 2.0 in stage 1.0 (TID 2, localhost, PROCESS_LOCAL, 1855 bytes)
14/07/15 19:44:40 INFO TaskSetManager: Starting task 3.0 in stage 1.0 (TID 3, localhost, PROCESS_LOCAL, 1855 bytes)
14/07/15 19:44:40 INFO TaskSetManager: Starting task 4.0 in stage 1.0 (TID 4, localhost, PROCESS_LOCAL, 1855 bytes)
14/07/15 19:44:40 INFO TaskSetManager: Starting task 5.0 in stage 1.0 (TID 5, localhost, PROCESS_LOCAL, 1855 bytes)
14/07/15 19:44:40 INFO TaskSetManager: Starting task 6.0 in stage 1.0 (TID 6, localhost, PROCESS_LOCAL, 1855 bytes)
...
14/07/15 19:44:40 INFO TaskSetManager: Finished task 1.0 in stage 1.0 (TID 1) in 64 ms on localhost (4/10)
14/07/15 19:44:40 INFO TaskSetManager: Finished task 4.0 in stage 1.0 (TID 4) in 63 ms on localhost (5/10)
14/07/15 19:44:40 INFO TaskSetManager: Finished task 2.0 in stage 1.0 (TID 2) in 63 ms on localhost (6/10)
14/07/15 19:44:40 INFO TaskSetManager: Finished task 7.0 in stage 1.0 (TID 7) in 62 ms on localhost (7/10)
14/07/15 19:44:40 INFO TaskSetManager: Finished task 6.0 in stage 1.0 (TID 6) in 63 ms on localhost (8/10)
14/07/15 19:44:40 INFO TaskSetManager: Finished task 9.0 in stage 1.0 (TID 9) in 8 ms on localhost (9/10)
14/07/15 19:44:40 INFO TaskSetManager: Finished task 8.0 in stage 1.0 (TID 8) in 9 ms on localhost (10/10)

```

Executor log looks like
```
14/07/15 19:44:40 INFO Executor: Running task 0.0 in stage 1.0 (TID 0)
14/07/15 19:44:40 INFO Executor: Running task 3.0 in stage 1.0 (TID 3)
14/07/15 19:44:40 INFO Executor: Running task 1.0 in stage 1.0 (TID 1)
14/07/15 19:44:40 INFO Executor: Running task 4.0 in stage 1.0 (TID 4)
14/07/15 19:44:40 INFO Executor: Running task 2.0 in stage 1.0 (TID 2)
14/07/15 19:44:40 INFO Executor: Running task 5.0 in stage 1.0 (TID 5)
14/07/15 19:44:40 INFO Executor: Running task 6.0 in stage 1.0 (TID 6)
14/07/15 19:44:40 INFO Executor: Running task 7.0 in stage 1.0 (TID 7)
14/07/15 19:44:40 INFO Executor: Finished task 3.0 in stage 1.0 (TID 3). 847 bytes result sent to driver
14/07/15 19:44:40 INFO Executor: Finished task 2.0 in stage 1.0 (TID 2). 847 bytes result sent to driver
14/07/15 19:44:40 INFO Executor: Finished task 0.0 in stage 1.0 (TID 0). 847 bytes result sent to driver
14/07/15 19:44:40 INFO Executor: Finished task 1.0 in stage 1.0 (TID 1). 847 bytes result sent to driver
14/07/15 19:44:40 INFO Executor: Finished task 5.0 in stage 1.0 (TID 5). 847 bytes result sent to driver
14/07/15 19:44:40 INFO Executor: Finished task 4.0 in stage 1.0 (TID 4). 847 bytes result sent to driver
14/07/15 19:44:40 INFO Executor: Finished task 6.0 in stage 1.0 (TID 6). 847 bytes result sent to driver
14/07/15 19:44:40 INFO Executor: Finished task 7.0 in stage 1.0 (TID 7). 847 bytes result sent to driver
```

Author: Reynold Xin <rxin@apache.org>

Closes #1259 from rxin/betterTaskLogging and squashes the following commits:

c28ada1 [Reynold Xin] Fix unit test failure.
987d043 [Reynold Xin] Updated log messages.
c6cfd46 [Reynold Xin] Merge branch 'master' into betterTaskLogging
b7b1bcc [Reynold Xin] Fixed a typo.
f9aba3c [Reynold Xin] Made it compile.
f8a5c06 [Reynold Xin] Merge branch 'master' into betterTaskLogging
07264e6 [Reynold Xin] Defensive check against unknown TaskEndReason.
76bbd18 [Reynold Xin] FailureSuite not serializable reporting.
4659b20 [Reynold Xin] Remove unused variable.
53888e3 [Reynold Xin] [SPARK-2317] Improve task logging.
2014-07-16 11:50:49 -07:00
Reynold Xin ef48222c10 [SPARK-2517] Remove some compiler warnings.
Author: Reynold Xin <rxin@apache.org>

Closes #1433 from rxin/compile-warning and squashes the following commits:

8d0b890 [Reynold Xin] Remove some compiler warnings.
2014-07-16 11:15:07 -07:00
Rui Li 33e64ecacb SPARK-2277: make TaskScheduler track hosts on rack
Hi mateiz, I've created [SPARK-2277](https://issues.apache.org/jira/browse/SPARK-2277) to make TaskScheduler track hosts on each rack. Please help to review, thanks.

Author: Rui Li <rui.li@intel.com>

Closes #1212 from lirui-intel/trackHostOnRack and squashes the following commits:

2b4bd0f [Rui Li] SPARK-2277: refine UT
fbde838 [Rui Li] SPARK-2277: add UT
7bbe658 [Rui Li] SPARK-2277: rename the method
5e4ef62 [Rui Li] SPARK-2277: remove unnecessary import
79ac750 [Rui Li] SPARK-2277: make TaskScheduler track hosts on rack
2014-07-16 22:53:37 +05:30
Reynold Xin 4576d80a51 [SPARK-2469] Use Snappy (instead of LZF) for default shuffle compression codec
This reduces shuffle compression memory usage by 3x.

Author: Reynold Xin <rxin@apache.org>

Closes #1415 from rxin/snappy and squashes the following commits:

06c1a01 [Reynold Xin] SPARK-2469: Use Snappy (instead of LZF) for default shuffle compression codec.
2014-07-15 18:47:39 -07:00
Reynold Xin dd95abada7 [SPARK-2399] Add support for LZ4 compression.
Based on Greg Bowyer's patch from JIRA https://issues.apache.org/jira/browse/SPARK-2399

Author: Reynold Xin <rxin@apache.org>

Closes #1416 from rxin/lz4 and squashes the following commits:

6c8fefe [Reynold Xin] Fixed typo.
8a14d38 [Reynold Xin] [SPARK-2399] Add support for LZ4 compression.
2014-07-15 01:46:57 -07:00
Daoyuan 38ccd6ebd4 move some test file to match src code
Just move some test suite to corresponding package

Author: Daoyuan <daoyuan.wang@intel.com>

Closes #1401 from adrian-wang/movetestfiles and squashes the following commits:

d1a6803 [Daoyuan] move some test file to match src code
2014-07-14 10:40:44 -07:00
Daniel Darabos 2245c87af4 Use the Executor's ClassLoader in sc.objectFile().
This makes it possible to read classes from the object file which were specified in the user-provided jars. (By default ObjectInputStream uses latestUserDefinedLoader, which may or may not be the right one.)

I created this because I ran into the following problem. I have x:RDD[X] with X being defined in the jar that I provide to SparkContext. I save it with x.saveAsObjectFile("x"). I try to load it with sc.objectFile\[X\]("x"). It fails with ClassNotFoundException.

After a good while of debugging I figured out that Utils.deserialize() most likely uses the ClassLoader of Utils. This is the bootstrap ClassLoader, so it is not aware of the dynamically added jars. This patch fixes the issue.

A more robust fix would be to always default to Thread.currentThread.getContextClassLoader. This would prevent this problem from biting anyone in the future. It would be a bit harder to test though. On the topic of testing, if you'd like to see tests for this, I will need some hand-holding. Thanks!

Author: Daniel Darabos <darabos.daniel@gmail.com>

Closes #181 from darabos/master and squashes the following commits:

45a011a [Daniel Darabos] Add test for SPARK-1877. (Fixed in 52eb54d.)
e13e090 [Daniel Darabos] Merge branch 'master' of https://github.com/apache/spark
61fe0d0 [Daniel Darabos] Fix style (line too long).
1b5df2c [Daniel Darabos] Use the Executor's ClassLoader in sc.objectFile(). This makes it possible to read classes from the object file which were specified in the user-provided jars. (By default ObjectInputStream uses latestUserDefinedLoader, which may or may not be the right one.)
2014-07-12 00:07:42 -07:00
witgo 3cd5029be7 Resolve sbt warnings during build Ⅱ
Author: witgo <witgo@qq.com>

Closes #1153 from witgo/expectResult and squashes the following commits:

97541d8 [witgo] merge master
ead26e7 [witgo] Resolve sbt warnings during build
2014-07-08 00:31:42 -07:00
Andrew Or 3894a49be9 [SPARK-2307][Reprise] Correctly report RDD blocks on SparkUI
**Problem.** The existing code in `ExecutorPage.scala` requires a linear scan through all the blocks to filter out the uncached ones. Every refresh could be expensive if there are many blocks and many executors.

**Solution.** The proper semantics should be the following: `StorageStatusListener` should contain only block statuses that are cached. This means as soon as a block is unpersisted by any mean, its status should be removed. This is reflected in the changes made in `StorageStatusListener.scala`.

Further, the `StorageTab` must stop relying on the `StorageStatusListener` changing a dropped block's status to `StorageLevel.NONE` (which no longer happens). This is reflected in the changes made in `StorageTab.scala` and `StorageUtils.scala`.

----------

If you have been following this chain of PRs like pwendell, you will quickly notice that this reverts the changes in #1249, which reverts the changes in #1080. In other words, we are adding back the changes from #1080, and fixing SPARK-2307 on top of those changes. Please ask questions if you are confused.

Author: Andrew Or <andrewor14@gmail.com>

Closes #1255 from andrewor14/storage-ui-fix-reprise and squashes the following commits:

45416fa [Andrew Or] Merge branch 'master' of github.com:apache/spark into storage-ui-fix-reprise
a82ea25 [Andrew Or] Add tests for StorageStatusListener
8773b01 [Andrew Or] Update comment / minor changes
3afde3f [Andrew Or] Correctly report the number of blocks on SparkUI
2014-07-03 22:48:23 -07:00
Andrew Or c480537739 [SPARK] Fix NPE for ExternalAppendOnlyMap
It did not handle null keys very gracefully before.

Author: Andrew Or <andrewor14@gmail.com>

Closes #1288 from andrewor14/fix-external and squashes the following commits:

312b8d8 [Andrew Or] Abstract key hash code
ed5adf9 [Andrew Or] Fix NPE for ExternalAppendOnlyMap
2014-07-03 10:26:50 -07:00
Kay Ousterhout 05c3d90e35 [SPARK-2185] Emit warning when task size exceeds a threshold.
This functionality was added in an earlier commit but shortly
after was removed due to a bad git merge (totally my fault).

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #1149 from kayousterhout/warning_bug and squashes the following commits:

3f1bb00 [Kay Ousterhout] Fixed Json tests
462a664 [Kay Ousterhout] Removed task set name from warning message
e89b2f6 [Kay Ousterhout] Fixed Json tests.
7af424c [Kay Ousterhout] Emit warning when task size exceeds a threshold.
2014-07-01 01:56:51 -07:00
Reynold Xin 358ae1534d [SPARK-2322] Exception in resultHandler should NOT crash DAGScheduler and shutdown SparkContext.
This should go into 1.0.1.

Author: Reynold Xin <rxin@apache.org>

Closes #1264 from rxin/SPARK-2322 and squashes the following commits:

c77c07f [Reynold Xin] Added comment to SparkDriverExecutionException and a test case for accumulator.
5d8d920 [Reynold Xin] [SPARK-2322] Exception in resultHandler could crash DAGScheduler and shutdown SparkContext.
2014-06-30 11:50:22 -07:00
William Benton a484030dae SPARK-897: preemptively serialize closures
These commits cause `ClosureCleaner.clean` to attempt to serialize the cleaned closure with the default closure serializer and throw a `SparkException` if doing so fails.  This behavior is enabled by default but can be disabled at individual callsites of `SparkContext.clean`.

Commit 98e01ae8 fixes some no-op assertions in `GraphSuite` that this work exposed; I'm happy to put that in a separate PR if that would be more appropriate.

Author: William Benton <willb@redhat.com>

Closes #143 from willb/spark-897 and squashes the following commits:

bceab8a [William Benton] Commented DStream corner cases for serializability checking.
64d04d2 [William Benton] FailureSuite now checks both messages and causes.
3b3f74a [William Benton] Stylistic and doc cleanups.
b215dea [William Benton] Fixed spurious failures in ImplicitOrderingSuite
be1ecd6 [William Benton] Don't check serializability of DStream transforms.
abe816b [William Benton] Make proactive serializability checking optional.
5bfff24 [William Benton] Adds proactive closure-serializablilty checking
ed2ccf0 [William Benton] Test cases for SPARK-897.
2014-06-29 23:27:34 -07:00
jerryshao 66135a341d [SPARK-2104] Fix task serializing issues when sort with Java non serializable class
Details can be see in [SPARK-2104](https://issues.apache.org/jira/browse/SPARK-2104). This work is based on Reynold's work, add some unit tests to validate the issue.

@rxin , would you please take a look at this PR, thanks a lot.

Author: jerryshao <saisai.shao@intel.com>

Closes #1245 from jerryshao/SPARK-2104 and squashes the following commits:

c8ee362 [jerryshao] Make field partitions transient
2b41917 [jerryshao] Minor changes
47d763c [jerryshao] Fix task serializing issue when sort with Java non serializable class
2014-06-29 23:00:00 -07:00
Kay Ousterhout 7b71a0e096 [SPARK-1683] Track task read metrics.
This commit adds a new metric in TaskMetrics to record
the input data size and displays this information in the UI.

An earlier version of this commit also added the read time,
which can be useful for diagnosing straggler problems,
but unfortunately that change introduced a significant performance
regression for jobs that don't do much computation. In order to
track read time, we'll need to do sampling.

The screenshots below show the UI with the new "Input" field,
which I added to the stage summary page, the executor summary page,
and the per-stage page.

![image](https://cloud.githubusercontent.com/assets/1108612/3167930/2627f92a-eb77-11e3-861c-98ea5bb7a1a2.png)

![image](https://cloud.githubusercontent.com/assets/1108612/3167936/475a889c-eb77-11e3-9706-f11c48751f17.png)

![image](https://cloud.githubusercontent.com/assets/1108612/3167948/80ebcf12-eb77-11e3-87ed-349fce6a770c.png)

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #962 from kayousterhout/read_metrics and squashes the following commits:

f13b67d [Kay Ousterhout] Correctly format input bytes on executor page
8b70cde [Kay Ousterhout] Added comment about potential inaccuracy of bytesRead
d1016e8 [Kay Ousterhout] Udated SparkListenerSuite test
8461492 [Kay Ousterhout] Miniscule style fix
ae04d99 [Kay Ousterhout] Remove input metrics for parallel collections
719f19d [Kay Ousterhout] Style fixes
bb6ec62 [Kay Ousterhout] Small fixes
869ac7b [Kay Ousterhout] Updated Json tests
44a0301 [Kay Ousterhout] Fixed accidentally added line
4bd0568 [Kay Ousterhout] Added input source, renamed Hdfs to Hadoop.
f27e535 [Kay Ousterhout] Updates based on review comments and to fix rebase
bf41029 [Kay Ousterhout] Updated Json tests to pass
0fc33e0 [Kay Ousterhout] Added explicit backward compatibility test
4e52925 [Kay Ousterhout] Added Json output and associated tests.
365400b [Kay Ousterhout] [SPARK-1683] Track task read metrics.
2014-06-29 22:01:42 -07:00
Xiangrui Meng c23f5db32b [SPARK-2251] fix concurrency issues in random sampler
The following code is very likely to throw an exception:

~~~
val rdd = sc.parallelize(0 until 111, 10).sample(false, 0.1)
rdd.zip(rdd).count()
~~~

because the same random number generator is used in compute partitions.

Author: Xiangrui Meng <meng@databricks.com>

Closes #1229 from mengxr/fix-sample and squashes the following commits:

f1ee3d7 [Xiangrui Meng] fix concurrency issues in random sampler
2014-06-26 21:46:55 -07:00
Reynold Xin d1636dd72f [SPARK-2297][UI] Make task attempt and speculation more explicit in UI.
New UI:

![screen shot 2014-06-26 at 1 43 52 pm](https://cloud.githubusercontent.com/assets/323388/3404643/82b9ddc6-fd73-11e3-96f9-f7592a7aee79.png)

Author: Reynold Xin <rxin@apache.org>

Closes #1236 from rxin/ui-task-attempt and squashes the following commits:

3b645dd [Reynold Xin] Expose attemptId in Stage.
c0474b1 [Reynold Xin] Beefed up unit test.
c404bdd [Reynold Xin] Fix ReplayListenerSuite.
f56be4b [Reynold Xin] Fixed JsonProtocolSuite.
e29e0f7 [Reynold Xin] Minor update.
5e4354a [Reynold Xin] [SPARK-2297][UI] Make task attempt and speculation more explicit in UI.
2014-06-26 21:13:26 -07:00
Reynold Xin bf578deaf2 Removed throwable field from FetchFailedException and added MetadataFetchFailedException
FetchFailedException used to have a Throwable field, but in reality we never propagate any of the throwable/exceptions back to the driver because Executor explicitly looks for FetchFailedException and then sends FetchFailed as the TaskEndReason.

This pull request removes the throwable and adds a MetadataFetchFailedException that extends FetchFailedException (so now MapOutputTracker throws MetadataFetchFailedException instead).

Author: Reynold Xin <rxin@apache.org>

Closes #1227 from rxin/metadataFetchException and squashes the following commits:

5cb1e0a [Reynold Xin] MetadataFetchFailedException extends FetchFailedException.
8861ee2 [Reynold Xin] Throw MetadataFetchFailedException in MapOutputTracker.
2014-06-26 21:12:16 -07:00
Reynold Xin 4a346e242c [SPARK-2284][UI] Mark all failed tasks as failures.
Previously only tasks failed with ExceptionFailure reason was marked as failure.

Author: Reynold Xin <rxin@apache.org>

Closes #1224 from rxin/SPARK-2284 and squashes the following commits:

be79dbd [Reynold Xin] [SPARK-2284][UI] Mark all failed tasks as failures.
2014-06-25 22:35:03 -07:00
Mark Hamstra b88a59a668 [SPARK-1749] Job cancellation when SchedulerBackend does not implement killTask
This is a fixed up version of #686 (cc @markhamstra @pwendell).  The last commit (the only one I authored) reflects the changes I made from Mark's original patch.

Author: Mark Hamstra <markhamstra@gmail.com>
Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #1219 from kayousterhout/mark-SPARK-1749 and squashes the following commits:

42dfa7e [Kay Ousterhout] Got rid of terrible double-negative name
80b3205 [Kay Ousterhout] Don't notify listeners of job failure if it wasn't successfully cancelled.
d156d33 [Mark Hamstra] Do nothing in no-kill submitTasks
9312baa [Mark Hamstra] code review update
cc353c8 [Mark Hamstra] scalastyle
e61f7f8 [Mark Hamstra] Catch UnsupportedOperationException when DAGScheduler tries to cancel a job on a SchedulerBackend that does not implement killTask
2014-06-25 20:57:48 -07:00
Reynold Xin 7ff2c754f3 [SPARK-2270] Kryo cannot serialize results returned by asJavaIterable
and thus groupBy/cogroup are broken in Java APIs when Kryo is used).

@pwendell this should be merged into 1.0.1.

Thanks @sorenmacbeth for reporting this & helping out with the fix.

Author: Reynold Xin <rxin@apache.org>

Closes #1206 from rxin/kryo-iterable-2270 and squashes the following commits:

09da0aa [Reynold Xin] Updated the comment.
009bf64 [Reynold Xin] [SPARK-2270] Kryo cannot serialize results returned by asJavaIterable (and thus groupBy/cogroup are broken in Java APIs when Kryo is used).
2014-06-25 12:43:22 -07:00
witgo b6b44853cd SPARK-2248: spark.default.parallelism does not apply in local mode
Author: witgo <witgo@qq.com>

Closes #1194 from witgo/SPARK-2248 and squashes the following commits:

6ac950b [witgo] spark.default.parallelism does not apply in local mode
2014-06-24 19:45:03 -07:00
Xiangrui Meng 8ca41769fb [SPARK-1112, 2156] Bootstrap to fetch the driver's Spark properties.
This is an alternative solution to #1124 . Before launching the executor backend, we first fetch driver's spark properties and use it to overwrite executor's spark properties. This should be better than #1124.

@pwendell Are there spark properties that might be different on the driver and on the executors?

Author: Xiangrui Meng <meng@databricks.com>

Closes #1132 from mengxr/akka-bootstrap and squashes the following commits:

77ff32d [Xiangrui Meng] organize imports
68e1dfb [Xiangrui Meng] use timeout from AkkaUtils; remove props from RegisteredExecutor
46d332d [Xiangrui Meng] fix a test
7947c18 [Xiangrui Meng] increase slack size for akka
4ab696a [Xiangrui Meng] bootstrap to retrieve driver spark conf
2014-06-24 19:06:07 -07:00
Kay Ousterhout 1978a9033e Fix broken Json tests.
The assertJsonStringEquals method was missing an "assert" so
did not actually check that the strings were equal. This commit
adds the missing assert and fixes subsequently revealed problems
with the JsonProtocolSuite.

@andrewor14 I changed some of the test functionality to match what it
looks like you intended based on the expected strings -- let me know if
anything here looks wrong.

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #1198 from kayousterhout/json_test_fix and squashes the following commits:

77f858f [Kay Ousterhout] Fix broken Json tests.
2014-06-24 16:54:50 -07:00
Rui Li 924b7082b1 SPARK-1937: fix issue with task locality
Don't check executor/host availability when creating a TaskSetManager. Because the executors may haven't been registered when the TaskSetManager is created, in which case all tasks will be considered "has no preferred locations", and thus losing data locality in later scheduling.

Author: Rui Li <rui.li@intel.com>
Author: lirui-intel <rui.li@intel.com>

Closes #892 from lirui-intel/delaySchedule and squashes the following commits:

8444d7c [Rui Li] fix code style
fafd57f [Rui Li] keep locality constraints within the valid levels
18f9e05 [Rui Li] restrict allowed locality
5b3fb2f [Rui Li] refine UT
99f843e [Rui Li] add unit test and fix bug
fff4123 [Rui Li] fix computing valid locality levels
685ed3d [Rui Li] remove delay shedule for pendingTasksWithNoPrefs
7b0177a [Rui Li] remove redundant code
c7b93b5 [Rui Li] revise patch
3d7da02 [lirui-intel] Update TaskSchedulerImpl.scala
cab4c71 [Rui Li] revised patch
539a578 [Rui Li] fix code style
cf0d6ac [Rui Li] fix code style
3dfae86 [Rui Li] re-compute pending tasks when new host is added
a225ac2 [Rui Li] SPARK-1937: fix issue with task locality
2014-06-24 11:40:37 -07:00
jerryshao 56eb8af187 [SPARK-2124] Move aggregation into shuffle implementations
This PR is a sub-task of SPARK-2044 to move the execution of aggregation into shuffle implementations.

I leave `CoGoupedRDD` and `SubtractedRDD` unchanged because they have their implementations of aggregation. I'm not sure is it suitable to change these two RDDs.

Also I do not move sort related code of `OrderedRDDFunctions` into shuffle, this will be solved in another sub-task.

Author: jerryshao <saisai.shao@intel.com>

Closes #1064 from jerryshao/SPARK-2124 and squashes the following commits:

4a05a40 [jerryshao] Modify according to comments
1f7dcc8 [jerryshao] Style changes
50a2fd6 [jerryshao] Fix test suite issue after moving aggregator to Shuffle reader and writer
1a96190 [jerryshao] Code modification related to the ShuffledRDD
308f635 [jerryshao] initial works of move combiner to ShuffleManager's reader and writer
2014-06-23 20:25:46 -07:00
Sean Owen 9fe28c35df SPARK-1316. Remove use of Commons IO
Commons IO is actually barely used, and is not a declared dependency. This just replaces with equivalents from the JDK and Guava.

Author: Sean Owen <sowen@cloudera.com>

Closes #1173 from srowen/SPARK-1316 and squashes the following commits:

2eb53db [Sean Owen] Reorder Guava import
8fde404 [Sean Owen] Remove use of Commons IO, which is not actually a dependency
2014-06-22 11:47:49 -07:00
Marcelo Vanzin 648553d48e Fix some tests.
- JavaAPISuite was trying to compare a bare path with a URI. Fix by
  extracting the path from the URI, since we know it should be a
  local path anyway/

- b9be1609 excluded the ASM dependency everywhere, but easymock needs
  it (because cglib needs it). So re-add the dependency, with test
  scope this time.

The second one above actually uncovered a weird situation: the maven
test target works, even though I can't find the class sbt complains
about in its classpath. sbt complains with:

  [error] Uncaught exception when running org.apache.spark.util
  .random.RandomSamplerSuite: java.lang.NoClassDefFoundError:
  org/objectweb/asm/Type

To avoid more weirdness caused by that, I explicitly added the asm
dependency to both maven and sbt (for tests only), and verified
the classes don't end up in the final assembly.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #917 from vanzin/flaky-tests and squashes the following commits:

d022320 [Marcelo Vanzin] Fix some tests.
2014-06-20 20:05:12 -07:00
Anant 010c460d62 [SPARK-2061] Made splits deprecated in JavaRDDLike
The jira for the issue can be found at: https://issues.apache.org/jira/browse/SPARK-2061
Most of spark has used over to consistently using `partitions` instead of `splits`. We should do likewise and add a `partitions` method to JavaRDDLike and have `splits` just call that. We should also go through all cases where other API's (e.g. Python) call `splits` and we should change those to use the newer API.

Author: Anant <anant.asty@gmail.com>

Closes #1062 from anantasty/SPARK-2061 and squashes the following commits:

b83ce6b [Anant] Fixed syntax issue
21f9210 [Anant] Fixed version number in deprecation string
9315b76 [Anant] made related changes to use partitions in python api
8c62dd1 [Anant] Made splits deprecated in JavaRDDLike
2014-06-20 18:57:24 -07:00
Doris Xin e99903b84a [SPARK-1970] Update unit test in XORShiftRandomSuite to use ChiSquareTest from commons-math3
Updating the chisquare unit test in XORShiftRandomSuite to use the ChiSquareTest in commons-math3 instead of hardcoding the chisquare statistic for the desired confidence interval.

Author: Doris Xin <doris.s.xin@gmail.com>

Closes #1073 from dorx/math3Unit and squashes the following commits:

da0e891 [Doris Xin] remove math3 from common pom
9954143 [Doris Xin] merge master
c19948f [Doris Xin] Merge branch 'master' into math3Unit
8f84f19 [Doris Xin] [SPARK-1970] unit test in XORShiftRandomSuite
ffea61a [Doris Xin] SPARK-1939: Refactor takeSample method in RDD
1441977 [Doris Xin] SPARK-1939 Refactor takeSample method in RDD to use ScaSRS
2014-06-20 18:42:02 -07:00
Allan Douglas R. de Oliveira 6a224c31e8 SPARK-1868: Users should be allowed to cogroup at least 4 RDDs
Adds cogroup for 4 RDDs.

Author: Allan Douglas R. de Oliveira <allandouglas@gmail.com>

Closes #813 from douglaz/more_cogroups and squashes the following commits:

f8d6273 [Allan Douglas R. de Oliveira] Test python groupWith for one more case
0e9009c [Allan Douglas R. de Oliveira] Added scala tests
c3ffcdd [Allan Douglas R. de Oliveira] Added java tests
517a67f [Allan Douglas R. de Oliveira] Added tests for python groupWith
2f402d5 [Allan Douglas R. de Oliveira] Removed TODO
17474f4 [Allan Douglas R. de Oliveira] Use new cogroup function
7877a2a [Allan Douglas R. de Oliveira] Fixed code
ba02414 [Allan Douglas R. de Oliveira] Added varargs cogroup to pyspark
c4a8a51 [Allan Douglas R. de Oliveira] Added java cogroup 4
e94963c [Allan Douglas R. de Oliveira] Fixed spacing
f1ee57b [Allan Douglas R. de Oliveira] Fixed scala style issues
d7196f1 [Allan Douglas R. de Oliveira] Allow the cogroup of 4 RDDs
2014-06-20 11:03:03 -07:00
Patrick Wendell e5514790d7 HOTFIX: SPARK-2208 local metrics tests can fail on fast machines
Author: Patrick Wendell <pwendell@gmail.com>

Closes #1141 from pwendell/hotfix and squashes the following commits:

83e4c79 [Patrick Wendell] HOTFIX: SPARK-2208 local metrics tests can fail on fast machines
2014-06-19 21:06:28 -07:00
Mark Hamstra 4cbeea83e0 SPARK-2158 Clean up core/stdout file from FileAppenderSuite
@tdas

Author: Mark Hamstra <markhamstra@gmail.com>

Closes #1100 from markhamstra/SPARK-2158 and squashes the following commits:

ae8e069 [Mark Hamstra] Response to TD's review
2f1e201 [Mark Hamstra] Cleanup 'stdout' file within FileAppenderSuite
2014-06-18 14:56:41 -07:00
Andrew Ash b92d16b114 SPARK-1063 Add .sortBy(f) method on RDD
This never got merged from the apache/incubator-spark repo (which is now deleted) but there had been several rounds of code review on this PR there.

I think this is ready for merging.

Author: Andrew Ash <andrew@andrewash.com>

This patch had conflicts when merged, resolved by
Committer: Reynold Xin <rxin@apache.org>

Closes #369 from ash211/sortby and squashes the following commits:

d09147a [Andrew Ash] Fix Ordering import
43d0a53 [Andrew Ash] Fix missing .collect()
29a54ed [Andrew Ash] Re-enable test by converting to a closure
5a95348 [Andrew Ash] Add license for RDDSuiteUtils
64ed6e3 [Andrew Ash] Remove leaked diff
d4de69a [Andrew Ash] Remove scar tissue
63638b5 [Andrew Ash] Add Python version of .sortBy()
45e0fde [Andrew Ash] Add Java version of .sortBy()
adf84c5 [Andrew Ash] Re-indent to keep line lengths under 100 chars
9d9b9d8 [Andrew Ash] Use parentheses on .collect() calls
0457b69 [Andrew Ash] Ignore failing test
99f0baf [Andrew Ash] Merge branch 'master' into sortby
222ae97 [Andrew Ash] Try moving Ordering objects out to a different class
3fd0dd3 [Andrew Ash] Add (failing) test for sortByKey with explicit Ordering
b8b5bbc [Andrew Ash] Align remove extra spaces that were used to align ='s in test code
8c53298 [Andrew Ash] Actually use ascending and numPartitions parameters
381eef2 [Andrew Ash] Correct silly typo
7db3e84 [Andrew Ash] Support ascending and numPartitions params in sortBy()
0f685fd [Andrew Ash] Merge remote-tracking branch 'origin/master' into sortby
ca4490d [Andrew Ash] Add .sortBy(f) method on RDD
2014-06-17 11:47:48 -07:00
Daniel Darabos 23a12ce20c SPARK-2035: Store call stack for stages, display it on the UI.
I'm not sure about the test -- I get a lot of unrelated failures for some reason. I'll try to sort it out. But hopefully the automation will test this for me if I send a pull request :).

I'll attach a demo HTML in [Jira](https://issues.apache.org/jira/browse/SPARK-2035).

Author: Daniel Darabos <darabos.daniel@gmail.com>
Author: Patrick Wendell <pwendell@gmail.com>

Closes #981 from darabos/darabos-call-stack and squashes the following commits:

f7c6bfa [Daniel Darabos] Fix bad merge. I undid 83c226d454 by Doris.
3d0a48d [Daniel Darabos] Merge remote-tracking branch 'upstream/master' into darabos-call-stack
b857849 [Daniel Darabos] Style: Break long line.
ecb5690 [Daniel Darabos] Include the last Spark method in the full stack trace. Otherwise it is not visible if the stage name is overridden.
d00a85b [Patrick Wendell] Make call sites for stages non-optional and well defined
b9eba24 [Daniel Darabos] Make StageInfo.details non-optional. Add JSON serialization code for the new field. Verify JSON backward compatibility.
4312828 [Daniel Darabos] Remove Mima excludes for CallSite. They should be unnecessary now, with SPARK-2070 fixed.
0920750 [Daniel Darabos] Merge remote-tracking branch 'upstream/master' into darabos-call-stack
a4b1faf [Daniel Darabos] Add Mima exclusions for the CallSite changes it has picked up. They are private methods/classes, so we ought to be safe.
932f810 [Daniel Darabos] Use empty CallSite instead of null in DAGSchedulerSuite. Outside of testing, this parameter always originates in SparkContext.scala, and will never be null.
ccd89d1 [Daniel Darabos] Fix long lines.
ac173e4 [Daniel Darabos] Hide "show details" if there are no details to show.
6182da6 [Daniel Darabos] Set a configurable limit on maximum call stack depth. It can be useful in memory-constrained situations with large numbers of stages.
8fe2e34 [Daniel Darabos] Store call stack for stages, display it on the UI.
2014-06-17 00:08:05 -07:00
Kan Zhang 7dd9fc67a6 [SPARK-1837] NumericRange should be partitioned in the same way as other...
... sequences

Author: Kan Zhang <kzhang@apache.org>

Closes #776 from kanzhang/SPARK-1837 and squashes the following commits:

e48f018 [Kan Zhang] [SPARK-1837] code refactoring
67c33b5 [Kan Zhang] minor change
403f9b1 [Kan Zhang] [SPARK-1837] NumericRange should be partitioned in the same way as other sequences
2014-06-14 14:31:28 -07:00
Doris Xin 1de1d703bf SPARK-1939 Refactor takeSample method in RDD to use ScaSRS
Modified the takeSample method in RDD to use the ScaSRS sampling technique to improve performance. Added a private method that computes sampling rate > sample_size/total to ensure sufficient sample size with success rate >= 0.9999. Added a unit test for the private method to validate choice of sampling rate.

Author: Doris Xin <doris.s.xin@gmail.com>
Author: dorx <doris.s.xin@gmail.com>
Author: Xiangrui Meng <meng@databricks.com>

Closes #916 from dorx/takeSample and squashes the following commits:

5b061ae [Doris Xin] merge master
444e750 [Doris Xin] edge cases
3de882b [dorx] Merge pull request #2 from mengxr/SPARK-1939
82dde31 [Xiangrui Meng] update pyspark's takeSample
48d954d [Doris Xin] remove unused imports from RDDSuite
fb1452f [Doris Xin] allowing num to be greater than count in all cases
1481b01 [Doris Xin] washing test tubes and making coffee
dc699f3 [Doris Xin] give back imports removed by accident in rdd.py
64e445b [Doris Xin] logwarnning as soon as it enters the while loop
55518ed [Doris Xin] added TODO for logging in rdd.py
eff89e2 [Doris Xin] addressed reviewer comments.
ecab508 [Doris Xin] "fixed checkstyle violation
0a9b3e3 [Doris Xin] "reviewer comment addressed"
f80f270 [Doris Xin] Merge branch 'master' into takeSample
ae3ad04 [Doris Xin] fixed edge cases to prevent overflow
065ebcd [Doris Xin] Merge branch 'master' into takeSample
9bdd36e [Doris Xin] Check sample size and move computeFraction
e3fd6a6 [Doris Xin] Merge branch 'master' into takeSample
7cab53a [Doris Xin] fixed import bug in rdd.py
ffea61a [Doris Xin] SPARK-1939: Refactor takeSample method in RDD
1441977 [Doris Xin] SPARK-1939 Refactor takeSample method in RDD to use ScaSRS
2014-06-12 19:44:27 -07:00
Doris Xin 83c226d454 [SPARK-2088] fix NPE in toString
After deserialization, the transient field creationSiteInfo does not get backfilled with the default value, but the toString method, which is invoked by the serializer, expects the field to always be non-null. An NPE is thrown when toString is called by the serializer when creationSiteInfo is null.

Author: Doris Xin <doris.s.xin@gmail.com>

Closes #1028 from dorx/toStringNPE and squashes the following commits:

f20021e [Doris Xin] unit test for toString after desrialization
6f0a586 [Doris Xin] Merge branch 'master' into toStringNPE
f47fecf [Doris Xin] Merge branch 'master' into toStringNPE
76199c6 [Doris Xin] [SPARK-2088] fix NPE in toString
2014-06-12 12:53:07 -07:00
Sandy Ryza ce92a9c18f SPARK-554. Add aggregateByKey.
Author: Sandy Ryza <sandy@cloudera.com>

Closes #705 from sryza/sandy-spark-554 and squashes the following commits:

2302b8f [Sandy Ryza] Add MIMA exclude
f52e0ad [Sandy Ryza] Fix Python tests for real
2f3afa3 [Sandy Ryza] Fix Python test
0b735e9 [Sandy Ryza] Fix line lengths
ae56746 [Sandy Ryza] Fix doc (replace T with V)
c2be415 [Sandy Ryza] Java and Python aggregateByKey
23bf400 [Sandy Ryza] SPARK-554.  Add aggregateByKey.
2014-06-12 08:14:25 -07:00
Matei Zaharia 508fd371d6 [SPARK-2044] Pluggable interface for shuffles
This is a first cut at moving shuffle logic behind a pluggable interface, as described at https://issues.apache.org/jira/browse/SPARK-2044, to let us more easily experiment with new shuffle implementations. It moves the existing shuffle code to a class HashShuffleManager behind a general ShuffleManager interface.

Two things are still missing to make this complete:
* MapOutputTracker needs to be hidden behind the ShuffleManager interface; this will also require adding methods to ShuffleManager that will let the DAGScheduler interact with it as it does with the MapOutputTracker today
* The code to do map-sides and reduce-side combine in ShuffledRDD, PairRDDFunctions, etc needs to be moved into the ShuffleManager's readers and writers

However, some of these may also be done later after we merge the current interface.

Author: Matei Zaharia <matei@databricks.com>

Closes #1009 from mateiz/pluggable-shuffle and squashes the following commits:

7a09862 [Matei Zaharia] review comments
be33d3f [Matei Zaharia] review comments
1513d4e [Matei Zaharia] Add ASF header
ac56831 [Matei Zaharia] Bug fix and better error message
4f681ba [Matei Zaharia] Move write part of ShuffleMapTask to ShuffleManager
f6f011d [Matei Zaharia] Move hash shuffle reader behind ShuffleManager interface
55c7717 [Matei Zaharia] Changed RDD code to use ShuffleReader
75cc044 [Matei Zaharia] Partial work to move hash shuffle in
2014-06-11 20:45:29 -07:00
witgo c48b6222ea Resolve scalatest warnings during build
Author: witgo <witgo@qq.com>

Closes #1032 from witgo/ShouldMatchers and squashes the following commits:

7ebf34c [witgo] Resolve scalatest warnings during build
2014-06-10 20:24:05 -07:00
Tathagata Das 4823bf470e [SPARK-1940] Enabling rolling of executor logs, and automatic cleanup of old executor logs
Currently, in the default log4j configuration, all the executor logs get sent to the file <code>[executor-working-dir]/stderr</code>. This does not all log files to be rolled, so old logs cannot be removed.

Using log4j RollingFileAppender allows log4j logs to be rolled, but all the logs get sent to a different set of files, other than the files <code>stdout</code> and <code>stderr</code> . So the logs are not visible in the Spark web UI any more as Spark web UI only reads the files <code>stdout</code> and <code>stderr</code>. Furthermore, it still does not allow the stdout and stderr to be cleared periodically in case a large amount of stuff gets written to them (e.g. by explicit `println` inside map function).

This PR solves this by implementing a simple `RollingFileAppender` within Spark (disabled by default). When enabled (using configuration parameter `spark.executor.rollingLogs.enabled`), the logs can get rolled over either by time interval (set with `spark.executor.rollingLogs.interval`, set to daily by default), or by size of logs (set with  `spark.executor.rollingLogs.size`). Finally, old logs can be automatically deleted by specifying how many of the latest log files to keep (set with `spark.executor.rollingLogs.keepLastN`).  The web UI has also been modified to show the logs across the rolled-over files.

You can test this locally (without waiting a whole day) by setting  configuration `spark.executor.rollingLogs.enabled=true` and `spark.executor.rollingLogs.interval=minutely`. Continuously generate logs by running spark jobs and the generated logs files would look like this (`stderr` and `stdout` are the most current log file that are being written to).

```
stderr
stderr--2014-05-27--14-37
stderr--2014-05-27--14-47
stderr--2014-05-27--15-05
stdout
stdout--2014-05-27--14-47
```

The web ui should show logs across these files.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #895 from tdas/rolling-logs and squashes the following commits:

fd8f87f [Tathagata Das] Minor change.
d326aee [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into rolling-logs
ad956c1 [Tathagata Das] Scala style fix.
1f0a6ec [Tathagata Das] Some more changes based on Patrick's PR comments.
c8bfe4e [Tathagata Das] Refactore FileAppender to a package spark.util.logging and broke up the file into multiple files. Changed configuration parameter names.
4224409 [Tathagata Das] Style fix.
108a9f8 [Tathagata Das] Added better constraint handling for rolling policies.
f7da977 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into rolling-logs
9134495 [Tathagata Das] Simplified rolling logs by removing Daily/Hourly/MinutelyRollingFileAppender, and removing the setting rollingLogs.enabled
312d874 [Tathagata Das] Minor fixes based on PR comments.
8a67d83 [Tathagata Das] Fixed comments.
b36cfd6 [Tathagata Das] Implemented RollingPolicy, TimeBasedRollingPolicy and SizeBasedRollingPolicy, and changed RollingFileAppender accordingly.
b7e8272 [Tathagata Das] Style fix,
374c9a9 [Tathagata Das] Added missing license.
24354ea [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into rolling-logs
6cc09c7 [Tathagata Das] Fixed bugs in rolling logs, and added more debug statements.
adf4910 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into rolling-logs
931f8fb [Tathagata Das] Changed log viewer in Spark web UI to handle rolling log files.
cb4fb6d [Tathagata Das] Added FileAppender and RollingFileAppender to generate rolling executor logs.
2014-06-10 20:22:02 -07:00
Kay Ousterhout 6cf335d79a Added a TaskSetManager unit test.
This test ensures that when there are no
alive executors that satisfy a particular locality level,
the TaskSetManager doesn't ever use that as the maximum
allowed locality level (this optimization ensures that a
job doesn't wait extra time in an attempt to satisfy
a scheduling locality level that is impossible).

@mateiz and @lirui-intel this unit test illustrates an issue
with #892 (it fails with that patch).

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #1024 from kayousterhout/scheduler_unit_test and squashes the following commits:

de6a08f [Kay Ousterhout] Added a TaskSetManager unit test.
2014-06-09 13:13:53 -07:00
witgo 41c4a33105 [SPARK-1841]: update scalatest to version 2.1.5
Author: witgo <witgo@qq.com>

Closes #713 from witgo/scalatest and squashes the following commits:

b627a6a [witgo] merge master
51fb3d6 [witgo] merge master
3771474 [witgo] fix RDDSuite
996d6f9 [witgo] fix TimeStampedWeakValueHashMap test
9dfa4e7 [witgo] merge bug
1479b22 [witgo] merge master
29b9194 [witgo] fix code style
022a7a2 [witgo] fix test dependency
a52c0fa [witgo] fix test dependency
cd8f59d [witgo] Merge branch 'master' of https://github.com/apache/spark into scalatest
046540d [witgo] fix RDDSuite.scala
2c543b9 [witgo] fix ReplSuite.scala
c458928 [witgo] update scalatest to version 2.1.5
2014-06-06 11:45:21 -07:00
Matei Zaharia b45c13e7d7 SPARK-2043: ExternalAppendOnlyMap doesn't always find matching keys
The current implementation reads one key with the next hash code as it finishes reading the keys with the current hash code, which may cause it to miss some matches of the next key. This can cause operations like join to give the wrong result when reduce tasks spill to disk and there are hash collisions, as values won't be matched together. This PR fixes it by not reading in that next key, using a peeking iterator instead.

Author: Matei Zaharia <matei@databricks.com>

Closes #986 from mateiz/spark-2043 and squashes the following commits:

0959514 [Matei Zaharia] Added unit test for having many hash collisions
892debb [Matei Zaharia] SPARK-2043: don't read a key with the next hash code in ExternalAppendOnlyMap, instead use a buffered iterator to only read values with the current hash code.
2014-06-05 23:01:48 -07:00
CodingCat 89cdbb087c SPARK-1677: allow user to disable output dir existence checking
https://issues.apache.org/jira/browse/SPARK-1677

For compatibility with older versions of Spark it would be nice to have an option `spark.hadoop.validateOutputSpecs` (default true)  for the user to disable the output directory existence checking

Author: CodingCat <zhunansjtu@gmail.com>

Closes #947 from CodingCat/SPARK-1677 and squashes the following commits:

7930f83 [CodingCat] miao
c0c0e03 [CodingCat] bug fix and doc update
5318562 [CodingCat] bug fix
13219b5 [CodingCat] allow user to disable output dir existence checking
2014-06-05 11:39:35 -07:00
Sean Owen d341b17c2a SPARK-1973. Add randomSplit to JavaRDD (with tests, and tidy Java tests)
I'd like to use randomSplit through the Java API, and would like to add a convenience wrapper for this method to JavaRDD. This is fairly trivial. (In fact, is the intent that JavaRDD not wrap every RDD method? and that sometimes users should just use JavaRDD.wrapRDD()?)

Along the way, I added tests for it, and also touched up the Java API test style and behavior. This is maybe the more useful part of this small change.

Author: Sean Owen <sowen@cloudera.com>
Author: Xiangrui Meng <meng@databricks.com>

This patch had conflicts when merged, resolved by
Committer: Xiangrui Meng <meng@databricks.com>

Closes #919 from srowen/SPARK-1973 and squashes the following commits:

148cb7b [Sean Owen] Some final Java test polish, while we are at it
1fc3f3e [Xiangrui Meng] more cleaning on Java 8 tests
9ebc57f [Sean Owen] Use accumulator instead of temp files to test foreach
5efb0be [Sean Owen] Add Java randomSplit, and unit tests (including for sample)
5dcc158 [Sean Owen] Simplified Java 8 test with new language features, and fixed the name of MLB's greatest team
91a1769 [Sean Owen] Touch up minor style issues in existing Java API suite test
2014-06-04 11:27:08 -07:00
Kan Zhang c402a4a685 [SPARK-1817] RDD.zip() should verify partition sizes for each partition
RDD.zip() will throw an exception if it finds partition sizes are not the same.

Author: Kan Zhang <kzhang@apache.org>

Closes #944 from kanzhang/SPARK-1817 and squashes the following commits:

c073848 [Kan Zhang] [SPARK-1817] Cosmetic updates
524c670 [Kan Zhang] [SPARK-1817] RDD.zip() should verify partition sizes for each partition
2014-06-03 22:47:18 -07:00
Reynold Xin 1faef149f7 SPARK-1941: Update streamlib to 2.7.0 and use HyperLogLogPlus instead of HyperLogLog.
I also corrected some errors made in the previous HLL count approximate API, including relativeSD wasn't really a measure for error (and we used it to test error bounds in test results).

Author: Reynold Xin <rxin@apache.org>

Closes #897 from rxin/hll and squashes the following commits:

4d83f41 [Reynold Xin] New error bound and non-randomness.
f154ea0 [Reynold Xin] Added a comment on the value bound for testing.
e367527 [Reynold Xin] One more round of code review.
41e649a [Reynold Xin] Update final mima list.
9e320c8 [Reynold Xin] Incorporate code review feedback.
e110d70 [Reynold Xin] Merge branch 'master' into hll
354deb8 [Reynold Xin] Added comment on the Mima exclude rules.
acaa524 [Reynold Xin] Added the right exclude rules in MimaExcludes.
6555bfe [Reynold Xin] Added a default method and re-arranged MimaExcludes.
1db1522 [Reynold Xin] Excluded util.SerializableHyperLogLog from MIMA check.
9221b27 [Reynold Xin] Merge branch 'master' into hll
88cfe77 [Reynold Xin] Updated documentation and restored the old incorrect API to maintain API compatibility.
1294be6 [Reynold Xin] Updated HLL+.
e7786cb [Reynold Xin] Merge branch 'master' into hll
c0ef0c2 [Reynold Xin] SPARK-1941: Update streamlib to 2.7.0 and use HyperLogLogPlus instead of HyperLogLog.
2014-06-03 18:37:40 -07:00
Syed Hashmi 7782a304ad [SPARK-1942] Stop clearing spark.driver.port in unit tests
stop resetting spark.driver.port in unit tests (scala, java and python).

Author: Syed Hashmi <shashmi@cloudera.com>
Author: CodingCat <zhunansjtu@gmail.com>

Closes #943 from syedhashmi/master and squashes the following commits:

885f210 [Syed Hashmi] Removing unnecessary file (created by mergetool)
b8bd4b5 [Syed Hashmi] Merge remote-tracking branch 'upstream/master'
b895e59 [Syed Hashmi] Revert "[SPARK-1784] Add a new partitioner"
57b6587 [Syed Hashmi] Revert "[SPARK-1784] Add a balanced partitioner"
1574769 [Syed Hashmi] [SPARK-1942] Stop clearing spark.driver.port in unit tests
4354836 [Syed Hashmi] Revert "SPARK-1686: keep schedule() calling in the main thread"
fd36542 [Syed Hashmi] [SPARK-1784] Add a balanced partitioner
6668015 [CodingCat] SPARK-1686: keep schedule() calling in the main thread
4ca94cc [Syed Hashmi] [SPARK-1784] Add a new partitioner
2014-06-03 12:04:47 -07:00
witgo 4dbb27b0cf [SPARK-1712]: TaskDescription instance is too big causes Spark to hang
Author: witgo <witgo@qq.com>

Closes #694 from witgo/SPARK-1712_new and squashes the following commits:

0f52483 [witgo] review commit
83ce29b [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1712_new
52e6752 [witgo] reset test SparkContext
63636b6 [witgo] review commit
44a59ee [witgo] review commit
3b6d48c [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1712_new
926bd6a [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1712_new
9a5cfad [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1712_new
03cc562 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1712_new
b0930b0 [witgo] review commit
b1174bd [witgo] merge master
f76679b [witgo] merge master
689495d [witgo] fix scala style bug
1d35c3c [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1712_new
062c182 [witgo] fix small bug for code style
0a428cf [witgo] add unit tests
158b2dc [witgo] review commit
4afe71d [witgo] review commit
9e4ffa7 [witgo] review commit
1d35c7d [witgo] fix hang
7965580 [witgo] fix Statement order
0e29eac [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1712_new
3ea1ca1 [witgo] remove duplicate serialize
743a7ad [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1712_new
86e2048 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1712_new
2a89adc [witgo] SPARK-1712: TaskDescription instance is too big causes Spark to hang
2014-05-28 15:57:05 -07:00
Zhen Peng 8d271c90fa SPARK-1929 DAGScheduler suspended by local task OOM
DAGScheduler does not handle local task OOM properly, and will wait for the job result forever.

Author: Zhen Peng <zhenpeng01@baidu.com>

Closes #883 from zhpengg/bugfix-dag-scheduler-oom and squashes the following commits:

76f7eda [Zhen Peng] remove redundant memory allocations
aa63161 [Zhen Peng] SPARK-1929 DAGScheduler suspended by local task OOM
2014-05-26 21:30:25 -07:00
Andrew Or 5081a0a9d4 [SPARK-1900 / 1918] PySpark on YARN is broken
If I run the following on a YARN cluster
```
bin/spark-submit sheep.py --master yarn-client
```
it fails because of a mismatch in paths: `spark-submit` thinks that `sheep.py` resides on HDFS, and balks when it can't find the file there. A natural workaround is to add the `file:` prefix to the file:
```
bin/spark-submit file:/path/to/sheep.py --master yarn-client
```
However, this also fails. This time it is because python does not understand URI schemes.

This PR fixes this by automatically resolving all paths passed as command line argument to `spark-submit` properly. This has the added benefit of keeping file and jar paths consistent across different cluster modes. For python, we strip the URI scheme before we actually try to run it.

Much of the code is originally written by @mengxr. Tested on YARN cluster. More tests pending.

Author: Andrew Or <andrewor14@gmail.com>

Closes #853 from andrewor14/submit-paths and squashes the following commits:

0bb097a [Andrew Or] Format path correctly before adding it to PYTHONPATH
323b45c [Andrew Or] Include --py-files on PYTHONPATH for pyspark shell
3c36587 [Andrew Or] Improve error messages (minor)
854aa6a [Andrew Or] Guard against NPE if user gives pathological paths
6638a6b [Andrew Or] Fix spark-shell jar paths after #849 went in
3bb0359 [Andrew Or] Update more comments (minor)
2a1f8a0 [Andrew Or] Update comments (minor)
6af2c77 [Andrew Or] Merge branch 'master' of github.com:apache/spark into submit-paths
a68c4d1 [Andrew Or] Handle Windows python file path correctly
427a250 [Andrew Or] Resolve paths properly for Windows
a591a4a [Andrew Or] Update tests for resolving URIs
6c8621c [Andrew Or] Move resolveURIs to Utils
db8255e [Andrew Or] Merge branch 'master' of github.com:apache/spark into submit-paths
f542dce [Andrew Or] Fix outdated tests
691c4ce [Andrew Or] Ignore special primary resource names
5342ac7 [Andrew Or] Add missing space in error message
02f77f3 [Andrew Or] Resolve command line arguments to spark-submit properly
2014-05-24 18:01:49 -07:00
Aaron Davidson f9f5fd5f4e Fix UISuite unit test that fails under Jenkins contention
Due to perhaps zombie processes on Jenkins, it seems that at least 10
Spark ports are in use. It also doesn't matter that the port increases
when used, it could in fact go down -- the only part that matters is
that it selects a different port rather than failing to bind.
Changed test to match this.

Thanks to @andrewor14 for helping diagnose this.

Author: Aaron Davidson <aaron@databricks.com>

Closes #857 from aarondav/tiny and squashes the following commits:

c199ec8 [Aaron Davidson] Fix UISuite unit test that fails under Jenkins contention
2014-05-22 15:11:05 -07:00
Andrew Or 7c79ef7d43 [Minor] Move JdbcRDDSuite to the correct package
It was in the wrong package

Author: Andrew Or <andrewor14@gmail.com>

Closes #839 from andrewor14/jdbc-suite and squashes the following commits:

f948c5a [Andrew Or] cache -> cache()
b215279 [Andrew Or] Move JdbcRDDSuite to the correct package
2014-05-21 01:25:10 -07:00
Andrew Or 69f750228f [SPARK-1769] Executor loss causes NPE race condition
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
2014-05-14 00:54:33 -07:00
Patrick Wendell 7bb9a521f3 Revert "[SPARK-1784] Add a new partitioner to allow specifying # of keys per partition"
This reverts commit 92cebada09.
2014-05-13 23:24:51 -07:00
Syed Hashmi 92cebada09 [SPARK-1784] Add a new partitioner to allow specifying # of keys per partition
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
2014-05-13 21:24:23 -07:00
Ye Xianjin 753b04dea4 [SPARK-1527] change rootDir*.getName to rootDir*.getAbsolutePath
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.
2014-05-13 19:03:51 -07:00
Andrew Or 5c0dafc2c8 [SPARK-1816] LiveListenerBus dies if a listener throws an exception
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
2014-05-13 18:32:32 -07:00
William Benton 16ffadcc4a SPARK-571: forbid return statements in cleaned closures
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
2014-05-13 13:45:23 -07:00
Sean Owen 7120a2979d SPARK-1798. Tests should clean up temp files
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
2014-05-12 14:16:19 -07:00
Patrick Wendell 7d9cc9214b SPARK-1770: Load balance elements when repartitioning.
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.
2014-05-11 17:11:55 -07:00
Andrew Or 83e0424d87 [SPARK-1774] Respect SparkSubmit --jars on YARN (client)
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
2014-05-10 20:58:02 -07:00
Sean Owen 2b7bd29eb6 SPARK-1789. Multiple versions of Netty dependencies cause FlumeStreamSuite 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
2014-05-10 20:50:40 -07:00
Matei Zaharia 7eefc9d2b3 SPARK-1708. Add a ClassTag on Serializer and things that depend on it
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
2014-05-10 12:10:24 -07:00
Andrew Or 8b78412994 [SPARK-1755] Respect SparkSubmit --name on YARN
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
2014-05-08 20:45:29 -07:00
Andrew Or c3f8b78c21 [SPARK-1745] Move interrupted flag from TaskContext constructor (minor)
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
2014-05-08 12:13:07 -07:00
Matei Zaharia 951a5d9398 [SPARK-1549] Add Python support to spark-submit
This PR updates spark-submit to allow submitting Python scripts (currently only with deploy-mode=client, but that's all that was supported before) and updates the PySpark code to properly find various paths, etc. One significant change is that we assume we can always find the Python files either from the Spark assembly JAR (which will happen with the Maven assembly build in make-distribution.sh) or from SPARK_HOME (which will exist in local mode even if you use sbt assembly, and should be enough for testing). This means we no longer need a weird hack to modify the environment for YARN.

This patch also updates the Python worker manager to run python with -u, which means unbuffered output (send it to our logs right away instead of waiting a while after stuff was written); this should simplify debugging.

In addition, it fixes https://issues.apache.org/jira/browse/SPARK-1709, setting the main class from a JAR's Main-Class attribute if not specified by the user, and fixes a few help strings and style issues in spark-submit.

In the future we may want to make the `pyspark` shell use spark-submit as well, but it seems unnecessary for 1.0.

Author: Matei Zaharia <matei@databricks.com>

Closes #664 from mateiz/py-submit and squashes the following commits:

15e9669 [Matei Zaharia] Fix some uses of path.separator property
051278c [Matei Zaharia] Small style fixes
0afe886 [Matei Zaharia] Add license headers
4650412 [Matei Zaharia] Add pyFiles to PYTHONPATH in executors, remove old YARN stuff, add tests
15f8e1e [Matei Zaharia] Set PYTHONPATH in PythonWorkerFactory in case it wasn't set from outside
47c0655 [Matei Zaharia] More work to make spark-submit work with Python:
d4375bd [Matei Zaharia] Clean up description of spark-submit args a bit and add Python ones
2014-05-06 15:12:35 -07:00
Andrew Or 394d8cb1c4 Add tests for FileLogger, EventLoggingListener, and ReplayListenerBus
Modifications to Spark core are limited to exposing functionality to test files + minor style fixes.
(728 / 769 lines are from tests)

Author: Andrew Or <andrewor14@gmail.com>

Closes #591 from andrewor14/event-log-tests and squashes the following commits:

2883837 [Andrew Or] Merge branch 'master' of github.com:apache/spark into event-log-tests
c3afcea [Andrew Or] Compromise
2d5daf8 [Andrew Or] Use temp directory provided by the OS rather than /tmp
2b52151 [Andrew Or] Remove unnecessary file delete + add a comment
62010fd [Andrew Or] More cleanup (renaming variables, updating comments etc)
ad2beff [Andrew Or] Clean up EventLoggingListenerSuite + modify a few comments
862e752 [Andrew Or] Merge branch 'master' of github.com:apache/spark into event-log-tests
e0ba2f8 [Andrew Or] Fix test failures caused by race condition in processing/mutating events
b990453 [Andrew Or] ReplayListenerBus suite - tests do not all pass yet
ab66a84 [Andrew Or] Tests for FileLogger + delete file after tests
187bb25 [Andrew Or] Formatting and renaming variables
769336f [Andrew Or] Merge branch 'master' of github.com:apache/spark into event-log-tests
5d38ffe [Andrew Or] Clean up EventLoggingListenerSuite + add comments
e12f4b1 [Andrew Or] Preliminary tests for EventLoggingListener (need major cleanup)
2014-05-01 21:42:06 -07:00
witgo 7d15058410 SPARK-1509: add zipWithIndex zipWithUniqueId methods to java api
Author: witgo <witgo@qq.com>

Closes #423 from witgo/zipWithIndex and squashes the following commits:

039ec04 [witgo] Merge branch 'master' of https://github.com/apache/spark into zipWithIndex
24d74c9 [witgo] review commit
763a5e4 [witgo] Merge branch 'master' of https://github.com/apache/spark into zipWithIndex
59747d1 [witgo] review commit
7bf4d06 [witgo] Merge branch 'master' of https://github.com/apache/spark into zipWithIndex
daa8f84 [witgo] review commit
4070613 [witgo] Merge branch 'master' of https://github.com/apache/spark into zipWithIndex
18e6c97 [witgo] java api zipWithIndex test
11e2e7f [witgo] add zipWithIndex zipWithUniqueId methods to java api
2014-04-29 11:30:47 -07:00
Patrick Wendell 9f7a095184 SPARK-1652: Remove incorrect deprecation warning in spark-submit
This is a straightforward fix.

Author: Patrick Wendell <pwendell@gmail.com>

This patch had conflicts when merged, resolved by
Committer: Patrick Wendell <pwendell@gmail.com>

Closes #578 from pwendell/spark-submit-yarn and squashes the following commits:

96027c7 [Patrick Wendell] Test fixes
b5be173 [Patrick Wendell] Review feedback
4ac9cac [Patrick Wendell] SPARK-1652: spark-submit for yarn prints warnings even though calling as expected
2014-04-28 18:14:59 -07:00
Patrick Wendell 6b3c6e5dd8 SPARK-1145: Memory mapping with many small blocks can cause JVM allocation failures
This includes some minor code clean-up as well. The main change is that small files are not memory mapped. There is a nicer way to write that code block using Scala's `Try` but to make it easy to back port and as simple as possible, I opted for the more explicit but less pretty format.

Author: Patrick Wendell <pwendell@gmail.com>

Closes #43 from pwendell/block-iter-logging and squashes the following commits:

1cff512 [Patrick Wendell] Small issue from merge.
49f6c269 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into block-iter-logging
4943351 [Patrick Wendell] Added a test and feedback on mateis review
a637a18 [Patrick Wendell] Review feedback and adding rewind() when reading byte buffers.
b76b95f [Patrick Wendell] Review feedback
4e1514e [Patrick Wendell] Don't memory map for small files
d238b88 [Patrick Wendell] Some logging and clean-up
2014-04-27 17:40:56 -07:00
Patrick Wendell aa9a7f5db7 SPARK-1606: Infer user application arguments instead of requiring --arg.
This modifies spark-submit to do something more like the Hadoop `jar`
command. Now we have the following syntax:

./bin/spark-submit [options] user.jar [user options]

Author: Patrick Wendell <pwendell@gmail.com>

Closes #563 from pwendell/spark-submit and squashes the following commits:

32241fc [Patrick Wendell] Review feedback
3adfb69 [Patrick Wendell] Small fix
bc48139 [Patrick Wendell] SPARK-1606: Infer user application arguments instead of requiring --arg.
2014-04-26 19:24:29 -07:00
CodingCat 027f1b85f9 SPARK-1235: manage the DAGScheduler EventProcessActor with supervisor and refactor the DAGScheduler with Akka
https://spark-project.atlassian.net/browse/SPARK-1235

In the current implementation, the running job will hang if the DAGScheduler crashes for some reason (eventProcessActor throws exception in receive() )

The reason is that the actor will automatically restart when the exception is thrown during the running but is not captured properly (Akka behaviour), and the JobWaiters are still waiting there for the completion of the tasks

In this patch, I refactored the DAGScheduler with Akka and manage the eventProcessActor with supervisor, so that upon the failure of a eventProcessActor, the supervisor will terminate the EventProcessActor and close the SparkContext

thanks for @kayousterhout and @markhamstra to give the hints in JIRA

Author: CodingCat <zhunansjtu@gmail.com>
Author: Xiangrui Meng <meng@databricks.com>
Author: Nan Zhu <CodingCat@users.noreply.github.com>

Closes #186 from CodingCat/SPARK-1235 and squashes the following commits:

a7fb0ee [CodingCat] throw Exception on failure of creating DAG
124d82d [CodingCat] blocking the constructor until event actor is ready
baf2d38 [CodingCat] fix the issue brought by non-blocking actorOf
35c886a [CodingCat] fix bug
82d08b3 [CodingCat] calling actorOf on system to ensure it is blocking
310a579 [CodingCat] style fix
cd02d9a [Nan Zhu] small fix
561cfbc [CodingCat] recover doCheckpoint
c048d0e [CodingCat] call submitWaitingStages for every event
a9eea039 [CodingCat] address Matei's comments
ac878ab [CodingCat] typo fix
5d1636a [CodingCat] re-trigger the test.....
9dfb033 [CodingCat] remove unnecessary changes
a7a2a97 [CodingCat] add StageCancelled message
fdf3b17 [CodingCat] just to retrigger the test......
089bc2f [CodingCat] address andrew's comments
228f4b0 [CodingCat] address comments from Mark
b68c1c7 [CodingCat] refactor DAGScheduler with Akka
810efd8 [Xiangrui Meng] akka solution
2014-04-25 16:04:48 -07:00
Mridul Muralidharan 968c0187a1 SPARK-1586 Windows build fixes
Unfortunately, this is not exhaustive - particularly hive tests still fail due to path issues.

Author: Mridul Muralidharan <mridulm80@apache.org>

This patch had conflicts when merged, resolved by
Committer: Matei Zaharia <matei@databricks.com>

Closes #505 from mridulm/windows_fixes and squashes the following commits:

ef12283 [Mridul Muralidharan] Move to org.apache.commons.lang3 for StringEscapeUtils. Earlier version was buggy appparently
cdae406 [Mridul Muralidharan] Remove leaked changes from > 2G fix branch
3267f4b [Mridul Muralidharan] Fix build failures
35b277a [Mridul Muralidharan] Fix Scalastyle failures
bc69d14 [Mridul Muralidharan] Change from hardcoded path separator
10c4d78 [Mridul Muralidharan] Use explicit encoding while using getBytes
1337abd [Mridul Muralidharan] fix classpath while running in windows
2014-04-24 20:48:33 -07:00
Andrew Or ee6f7e22a4 [SPARK-1615] Synchronize accesses to the LiveListenerBus' event queue
Original poster is @zsxwing, who reported this bug in #516.

Much of SparkListenerSuite relies on LiveListenerBus's `waitUntilEmpty()` method. As the name suggests, this waits until the event queue is empty. However, the following race condition could happen:

(1) We dequeue an event
(2) The queue is empty, we return true (even though the event has not been processed)
(3) The test asserts something assuming that all listeners have finished executing (and fails)
(4) The listeners receive and process the event

This PR makes (1) and (4) atomic by synchronizing around it. To do that, however, we must avoid using `eventQueue.take`, which is blocking and will cause a deadlock if we synchronize around it. As a workaround, we use the non-blocking `eventQueue.poll` + a semaphore to provide the same semantics.

This has been a possible race condition for a long time, but for some reason we've never run into it.

Author: Andrew Or <andrewor14@gmail.com>

Closes #544 from andrewor14/stage-info-test-fix and squashes the following commits:

3cbe40c [Andrew Or] Merge github.com:apache/spark into stage-info-test-fix
56dbbcb [Andrew Or] Check if event is actually added before releasing semaphore
eb486ae [Andrew Or] Synchronize accesses to the LiveListenerBus' event queue
2014-04-24 20:18:15 -07:00
Thomas Graves 44da5ab2de Spark 1489 Fix the HistoryServer view acls
This allows the view acls set by the user to be enforced by the history server.  It also fixes filters being applied properly.

Author: Thomas Graves <tgraves@apache.org>

Closes #509 from tgravescs/SPARK-1489 and squashes the following commits:

869c186 [Thomas Graves] change to either acls enabled or disabled
0d8333c [Thomas Graves] Add history ui policy to allow acls to either use application set, history server force acls on, or off
65148b5 [Thomas Graves] SPARK-1489 Fix the HistoryServer view acls
2014-04-24 18:38:10 -07:00
Arun Ramakrishnan 35e3d199f0 SPARK-1438 RDD.sample() make seed param optional
copying form previous pull request https://github.com/apache/spark/pull/462

Its probably better to let the underlying language implementation take care of the default . This was easier to do with python as the default value for seed in random and numpy random is None.

In Scala/Java side it might mean propagating an Option or null(oh no!) down the chain until where the Random is constructed. But, looks like the convention in some other methods was to use System.nanoTime. So, followed that convention.

Conflict with overloaded method in sql.SchemaRDD.sample which also defines default params.
sample(fraction, withReplacement=false, seed=math.random)
Scala does not allow more than one overloaded to have default params. I believe the author intended to override the RDD.sample method and not overload it. So, changed it.

If backward compatible is important, 3 new method can be introduced (without default params) like this
sample(fraction)
sample(fraction, withReplacement)
sample(fraction, withReplacement, seed)

Added some tests for the scala RDD takeSample method.

Author: Arun Ramakrishnan <smartnut007@gmail.com>

This patch had conflicts when merged, resolved by
Committer: Matei Zaharia <matei@databricks.com>

Closes #477 from smartnut007/master and squashes the following commits:

07bb06e [Arun Ramakrishnan] SPARK-1438 fixing more space formatting issues
b9ebfe2 [Arun Ramakrishnan] SPARK-1438 removing redundant import of random in python rddsampler
8d05b1a [Arun Ramakrishnan] SPARK-1438 RDD . Replace System.nanoTime with a Random generated number. python: use a separate instance of Random instead of seeding language api global Random instance.
69619c6 [Arun Ramakrishnan] SPARK-1438 fix spacing issue
0c247db [Arun Ramakrishnan] SPARK-1438 RDD language apis to support optional seed in RDD methods sample/takeSample
2014-04-24 17:27:16 -07:00
Sandeep a03ac222d8 Fix Scala Style
Any comments are welcome

Author: Sandeep <sandeep@techaddict.me>

Closes #531 from techaddict/stylefix-1 and squashes the following commits:

7492730 [Sandeep] Pass 4
98b2428 [Sandeep] fix rxin suggestions
b5e2e6f [Sandeep] Pass 3
05932d7 [Sandeep] fix if else styling 2
08690e5 [Sandeep] fix if else styling
2014-04-24 15:07:23 -07:00
Reynold Xin 1fdf659d2f SPARK-1601 & SPARK-1602: two bug fixes related to cancellation
This should go into 1.0 since it would return wrong data when the bug happens (which is pretty likely if cancellation is used). Test case attached.

1. Do not put partially executed partitions into cache (in task killing).

2. Iterator returned by CacheManager#getOrCompute was not an InterruptibleIterator, and was thus leading to uninterruptible jobs.

Thanks @aarondav and @ahirreddy for reporting and helping debug.

Author: Reynold Xin <rxin@apache.org>

Closes #521 from rxin/kill and squashes the following commits:

401033f [Reynold Xin] Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/spark into kill
7a7bdd2 [Reynold Xin] Add a new line in the end of JobCancellationSuite.scala.
35cd9f7 [Reynold Xin] Fixed a bug that partially executed partitions can be put into cache (in task killing).
2014-04-24 00:27:45 -07:00
Mridul Muralidharan dd681f502e SPARK-1587 Fix thread leak
mvn test fails (intermittently) due to thread leak - since scalatest runs all tests in same vm.

Author: Mridul Muralidharan <mridulm80@apache.org>

Closes #504 from mridulm/resource_leak_fixes and squashes the following commits:

a5d10d0 [Mridul Muralidharan] Prevent thread leaks while running tests : cleanup all threads when SparkContext.stop is invoked. Causes tests to fail
7b5e19c [Mridul Muralidharan] Prevent NPE while running tests
2014-04-23 23:20:55 -07:00
Aaron Davidson d485eecb72 Update Java api for setJobGroup with interruptOnCancel
Also adds a unit test.

Author: Aaron Davidson <aaron@databricks.com>

Closes #522 from aarondav/cancel2 and squashes the following commits:

565c253 [Aaron Davidson] Update Java api for setJobGroup with interruptOnCancel
65b33d8 [Aaron Davidson] Add unit test for Thread interruption on cancellation
2014-04-23 22:00:22 -07:00
Andrew Or 4b2bab1d08 [Hot Fix #469] Fix flaky test in SparkListenerSuite
The two modified tests may fail if the race condition does not bid in our favor...

Author: Andrew Or <andrewor14@gmail.com>

Closes #516 from andrewor14/stage-info-test-fix and squashes the following commits:

b4b6100 [Andrew Or] Add/replace missing waitUntilEmpty() calls to listener bus
2014-04-23 21:59:33 -07:00
Matei Zaharia 640f9a0efe [SPARK-1540] Add an optional Ordering parameter to PairRDDFunctions.
In https://issues.apache.org/jira/browse/SPARK-1540 we'd like to look at Spark's API to see if we can take advantage of Comparable keys in more places, which will make external spilling more efficient. This PR is a first step towards that that shows how to pass an Ordering when available and still continue functioning otherwise. It does this using a new implicit parameter with a default value of null.

The API is currently only in Scala -- in Java we'd have to add new versions of mapToPair and such that take a Comparator, or a new method to add a "type hint" to an RDD. We can address those later though.

Unfortunately requiring all keys to be Comparable would not work without requiring RDDs in general to contain only Comparable types. The reason is that methods such as distinct() and intersection() do a shuffle, but should be usable on RDDs of any type. So ordering will have to remain an optimization for the types that can be ordered. I think this isn't a horrible outcome though because one of the nice things about Spark's API is that it works on objects of *any* type, without requiring you to specify a schema or implement Writable or stuff like that.

Author: Matei Zaharia <matei@databricks.com>

This patch had conflicts when merged, resolved by
Committer: Reynold Xin <rxin@apache.org>

Closes #487 from mateiz/ordered-keys and squashes the following commits:

bd565f6 [Matei Zaharia] Pass an Ordering to only one version of groupBy because the Scala language spec doesn't allow having an optional parameter on all of them (this was only compiling in Scala 2.10 due to a bug).
4629965 [Matei Zaharia] Add tests for other versions of groupBy
3beae85 [Matei Zaharia] Added a test for implicit orderings
80b7a3b [Matei Zaharia] Add an optional Ordering parameter to PairRDDFunctions.
2014-04-23 17:03:54 -07:00
Aaron Davidson 432201c7ee SPARK-1582 Invoke Thread.interrupt() when cancelling jobs
Sometimes executor threads are blocked waiting for IO or monitors, and the current implementation of job cancellation may never recover these threads. By simply invoking Thread.interrupt() during cancellation, we can often safely unblock the threads and use them for subsequent work.

Note that this feature must remain optional for now because of a bug in HDFS where Thread.interrupt() may cause nodes to be marked as permanently dead (as the InterruptedException is reinterpreted as an IOException during communication with some node).

Author: Aaron Davidson <aaron@databricks.com>

Closes #498 from aarondav/cancel and squashes the following commits:

e52b829 [Aaron Davidson] Don't use job.properties when null
82f78bb [Aaron Davidson] Update DAGSchedulerSuite
b67f472 [Aaron Davidson] Add comment on why interruptOnCancel is in setJobGroup
4cb9fd6 [Aaron Davidson] SPARK-1582 Invoke Thread.interrupt() when cancelling jobs
2014-04-23 16:52:49 -07:00
Xiangrui Meng 662c860ebc [FIX: SPARK-1376] use --arg instead of --args in SparkSubmit to avoid warning messages
Even if users use `--arg`, `SparkSubmit` still uses `--args` for child args internally, which triggers a warning message that may confuse users:

~~~
--args is deprecated. Use --arg instead.
~~~

@sryza Does it look good to you?

Author: Xiangrui Meng <meng@databricks.com>

Closes #485 from mengxr/submit-arg and squashes the following commits:

5e1b9fe [Xiangrui Meng] update test
cebbeb7 [Xiangrui Meng] use --arg instead of --args in SparkSubmit to avoid warning messages
2014-04-22 19:38:27 -07:00
Andrew Or 2de573877f [Spark-1538] Fix SparkUI incorrectly hiding persisted RDDs
**Bug**: After the following command `sc.parallelize(1 to 1000).persist.map(_ + 1).count()` is run, the the persisted RDD is missing from the storage tab of the SparkUI.

**Cause**: The command creates two RDDs in one stage, a `ParallelCollectionRDD` and a `MappedRDD`. However, the existing StageInfo only keeps the RDDInfo of the last RDD associated with the stage (`MappedRDD`), and so all RDD information regarding the first RDD (`ParallelCollectionRDD`) is discarded. In this case, we persist the first RDD,  but the StorageTab doesn't know about this RDD because it is not encoded in the StageInfo.

**Fix**: Record information of all RDDs in StageInfo, instead of just the last RDD (i.e. `stage.rdd`). Since stage boundaries are marked by shuffle dependencies, the solution is to traverse the last RDD's dependency tree, visiting only ancestor RDDs related through a sequence of narrow dependencies.

---

This PR also moves RDDInfo to its own file, includes a few style fixes, and adds a unit test for constructing StageInfos.

Author: Andrew Or <andrewor14@gmail.com>

Closes #469 from andrewor14/storage-ui-fix and squashes the following commits:

07fc7f0 [Andrew Or] Add back comment that was accidentally removed (minor)
5d799fe [Andrew Or] Add comment to justify testing of getNarrowAncestors with cycles
9d0e2b8 [Andrew Or] Hide details of getNarrowAncestors from outsiders
d2bac8a [Andrew Or] Deal with cycles in RDD dependency graph + add extensive tests
2acb177 [Andrew Or] Move getNarrowAncestors to RDD.scala
bfe83f0 [Andrew Or] Backtrace RDD dependency tree to find all RDDs that belong to a Stage
2014-04-22 19:24:03 -07:00
Andrew Or af46f1fd02 [Hot Fix] Ignore org.apache.spark.ui.UISuite tests
#446 faced a connection refused exception from these tests, causing them to timeout and fail after a long time. For now, let's disable these tests.

(We recently disabled the corresponding test in streaming in 7863ecca35. These tests are very similar).

Author: Andrew Or <andrewor14@gmail.com>

Closes #466 from andrewor14/ignore-ui-tests and squashes the following commits:

6f5a362 [Andrew Or] Ignore org.apache.spark.ui.UISuite tests
2014-04-21 12:37:43 -07:00
Patrick Wendell fb98488fc8 Clean up and simplify Spark configuration
Over time as we've added more deployment modes, this have gotten a bit unwieldy with user-facing configuration options in Spark. Going forward we'll advise all users to run `spark-submit` to launch applications. This is a WIP patch but it makes the following improvements:

1. Improved `spark-env.sh.template` which was missing a lot of things users now set in that file.
2. Removes the shipping of SPARK_CLASSPATH, SPARK_JAVA_OPTS, and SPARK_LIBRARY_PATH to the executors on the cluster. This was an ugly hack. Instead it introduces config variables spark.executor.extraJavaOpts, spark.executor.extraLibraryPath, and spark.executor.extraClassPath.
3. Adds ability to set these same variables for the driver using `spark-submit`.
4. Allows you to load system properties from a `spark-defaults.conf` file when running `spark-submit`. This will allow setting both SparkConf options and other system properties utilized by `spark-submit`.
5. Made `SPARK_LOCAL_IP` an environment variable rather than a SparkConf property. This is more consistent with it being set on each node.

Author: Patrick Wendell <pwendell@gmail.com>

Closes #299 from pwendell/config-cleanup and squashes the following commits:

127f301 [Patrick Wendell] Improvements to testing
a006464 [Patrick Wendell] Moving properties file template.
b4b496c [Patrick Wendell] spark-defaults.properties -> spark-defaults.conf
0086939 [Patrick Wendell] Minor style fixes
af09e3e [Patrick Wendell] Mention config file in docs and clean-up docs
b16e6a2 [Patrick Wendell] Cleanup of spark-submit script and Scala quick start guide
af0adf7 [Patrick Wendell] Automatically add user jar
a56b125 [Patrick Wendell] Responses to Tom's review
d50c388 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup
a762901 [Patrick Wendell] Fixing test failures
ffa00fe [Patrick Wendell] Review feedback
fda0301 [Patrick Wendell] Note
308f1f6 [Patrick Wendell] Properly escape quotes and other clean-up for YARN
e83cd8f [Patrick Wendell] Changes to allow re-use of test applications
be42f35 [Patrick Wendell] Handle case where SPARK_HOME is not set
c2a2909 [Patrick Wendell] Test compile fixes
4ee6f9d [Patrick Wendell] Making YARN doc changes consistent
afc9ed8 [Patrick Wendell] Cleaning up line limits and two compile errors.
b08893b [Patrick Wendell] Additional improvements.
ace4ead [Patrick Wendell] Responses to review feedback.
b72d183 [Patrick Wendell] Review feedback for spark env file
46555c1 [Patrick Wendell] Review feedback and import clean-ups
437aed1 [Patrick Wendell] Small fix
761ebcd [Patrick Wendell] Library path and classpath for drivers
7cc70e4 [Patrick Wendell] Clean up terminology inside of spark-env script
5b0ba8e [Patrick Wendell] Don't ship executor envs
84cc5e5 [Patrick Wendell] Small clean-up
1f75238 [Patrick Wendell] SPARK_JAVA_OPTS --> SPARK_MASTER_OPTS for master settings
4982331 [Patrick Wendell] Remove SPARK_LIBRARY_PATH
6eaf7d0 [Patrick Wendell] executorJavaOpts
0faa3b6 [Patrick Wendell] Stash of adding config options in submit script and YARN
ac2d65e [Patrick Wendell] Change spark.local.dir -> SPARK_LOCAL_DIRS
2014-04-21 10:26:33 -07:00
Kan Zhang 38877ccf39 Fixing a race condition in event listener unit test
Author: Kan Zhang <kzhang@apache.org>

Closes #401 from kanzhang/fix-1475 and squashes the following commits:

c6058bd [Kan Zhang] Fixing a race condition in event listener unit test
2014-04-16 17:39:11 -07:00
Holden Karau c3527a333a SPARK-1310: Start adding k-fold cross validation to MLLib [adds kFold to MLUtils & fixes bug in BernoulliSampler]
Author: Holden Karau <holden@pigscanfly.ca>

Closes #18 from holdenk/addkfoldcrossvalidation and squashes the following commits:

208db9b [Holden Karau] Fix a bad space
e84f2fc [Holden Karau] Fix the test, we should be looking at the second element instead
6ddbf05 [Holden Karau] swap training and validation order
7157ae9 [Holden Karau] CR feedback
90896c7 [Holden Karau] New line
150889c [Holden Karau] Fix up error messages in the MLUtilsSuite
2cb90b3 [Holden Karau] Fix the names in kFold
c702a96 [Holden Karau] Fix imports in MLUtils
e187e35 [Holden Karau] Move { up to same line as whenExecuting(random) in RandomSamplerSuite.scala
c5b723f [Holden Karau] clean up
7ebe4d5 [Holden Karau] CR feedback, remove unecessary learners (came back during merge mistake) and insert an empty line
bb5fa56 [Holden Karau] extra line sadness
163c5b1 [Holden Karau] code review feedback 1.to -> 1 to and folds -> numFolds
5a33f1d [Holden Karau] Code review follow up.
e8741a7 [Holden Karau] CR feedback
b78804e [Holden Karau] Remove cross validation [TODO in another pull request]
91eae64 [Holden Karau] Consolidate things in mlutils
264502a [Holden Karau] Add a test for the bug that was found with BernoulliSampler not copying the complement param
dd0b737 [Holden Karau] Wrap long lines (oops)
c0b7fa4 [Holden Karau] Switch FoldedRDD to use BernoulliSampler and PartitionwiseSampledRDD
08f8e4d [Holden Karau] Fix BernoulliSampler to respect complement
a751ec6 [Holden Karau] Add k-fold cross validation to MLLib
2014-04-16 09:33:27 -07:00
Sean Owen 0247b5c546 SPARK-1488. Resolve scalac feature warnings during build
For your consideration: scalac currently notes a number of feature warnings during compilation:

```
[warn] there were 65 feature warning(s); re-run with -feature for details
```

Warnings are like:

```
[warn] /Users/srowen/Documents/spark/core/src/main/scala/org/apache/spark/SparkContext.scala:1261: implicit conversion method rddToPairRDDFunctions should be enabled
[warn] by making the implicit value scala.language.implicitConversions visible.
[warn] This can be achieved by adding the import clause 'import scala.language.implicitConversions'
[warn] or by setting the compiler option -language:implicitConversions.
[warn] See the Scala docs for value scala.language.implicitConversions for a discussion
[warn] why the feature should be explicitly enabled.
[warn]   implicit def rddToPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) =
[warn]                ^
```

scalac is suggesting that it's just best practice to explicitly enable certain language features by importing them where used.

This PR simply adds the imports it suggests (and squashes one other Java warning along the way). This leaves just deprecation warnings in the build.

Author: Sean Owen <sowen@cloudera.com>

Closes #404 from srowen/SPARK-1488 and squashes the following commits:

8598980 [Sean Owen] Quiet scalac warnings about language features by explicitly importing language features.
39bc831 [Sean Owen] Enable -feature in scalac to emit language feature warnings
2014-04-14 19:50:00 -07:00
Xusen Yin 037fe4d2ba [SPARK-1415] Hadoop min split for wholeTextFiles()
JIRA issue [here](https://issues.apache.org/jira/browse/SPARK-1415).

New Hadoop API of `InputFormat` does not provide the `minSplits` parameter, which makes the API incompatible between `HadoopRDD` and `NewHadoopRDD`. The PR is for constructing compatible APIs.

Though `minSplits` is deprecated by New Hadoop API, we think it is better to make APIs compatible here.

**Note** that `minSplits` in `wholeTextFiles` could only be treated as a *suggestion*, the real number of splits may not be greater than `minSplits` due to `isSplitable()=false`.

Author: Xusen Yin <yinxusen@gmail.com>

Closes #376 from yinxusen/hadoop-min-split and squashes the following commits:

76417f6 [Xusen Yin] refine comments
c10af60 [Xusen Yin] refine comments and rewrite new class for wholeTextFile
766d05b [Xusen Yin] refine Java API and comments
4875755 [Xusen Yin] add minSplits for WholeTextFiles
2014-04-13 13:18:52 -07:00
Patrick Wendell 4bc07eebbf SPARK-1480: Clean up use of classloaders
The Spark codebase is a bit fast-and-loose when accessing classloaders and this has caused a few bugs to surface in master.

This patch defines some utility methods for accessing classloaders. This makes the intention when accessing a classloader much more explicit in the code and fixes a few cases where the wrong one was chosen.

case (a) -> We want the classloader that loaded Spark
case (b) -> We want the context class loader, or if not present, we want (a)

This patch provides a better fix for SPARK-1403 (https://issues.apache.org/jira/browse/SPARK-1403) than the current work around, which it reverts. It also fixes a previously unreported bug that the `./spark-submit` script did not work for running with `local` master. It didn't work because the executor classloader did not properly delegate to the context class loader (if it is defined) and in local mode the context class loader is set by the `./spark-submit` script. A unit test is added for that case.

Author: Patrick Wendell <pwendell@gmail.com>

Closes #398 from pwendell/class-loaders and squashes the following commits:

b4a1a58 [Patrick Wendell] Minor clean up
14f1272 [Patrick Wendell] SPARK-1480: Clean up use of classloaders
2014-04-13 08:58:37 -07:00
Tathagata Das 6aa08c39cf [SPARK-1386] Web UI for Spark Streaming
When debugging Spark Streaming applications it is necessary to monitor certain metrics that are not shown in the Spark application UI. For example, what is average processing time of batches? What is the scheduling delay? Is the system able to process as fast as it is receiving data? How many records I am receiving through my receivers?

While the StreamingListener interface introduced in the 0.9 provided some of this information, it could only be accessed programmatically. A UI that shows information specific to the streaming applications is necessary for easier debugging. This PR introduces such a UI. It shows various statistics related to the streaming application. Here is a screenshot of the UI running on my local machine.

http://i.imgur.com/1ooDGhm.png

This UI is integrated into the Spark UI running at 4040.

Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Andrew Or <andrewor14@gmail.com>

Closes #290 from tdas/streaming-web-ui and squashes the following commits:

fc73ca5 [Tathagata Das] Merge pull request #9 from andrewor14/ui-refactor
642dd88 [Andrew Or] Merge SparkUISuite.scala into UISuite.scala
eb30517 [Andrew Or] Merge github.com:apache/spark into ui-refactor
f4f4cbe [Tathagata Das] More minor fixes.
34bb364 [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui
252c566 [Tathagata Das] Merge pull request #8 from andrewor14/ui-refactor
e038b4b [Tathagata Das] Addressed Patrick's comments.
125a054 [Andrew Or] Disable serving static resources with gzip
90feb8d [Andrew Or] Address Patrick's comments
89dae36 [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui
72fe256 [Tathagata Das] Merge pull request #6 from andrewor14/ui-refactor
2fc09c8 [Tathagata Das] Added binary check exclusions
aa396d4 [Andrew Or] Rename tabs and pages (No more IndexPage.scala)
f8e1053 [Tathagata Das] Added Spark and Streaming UI unit tests.
caa5e05 [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui
585cd65 [Tathagata Das] Merge pull request #5 from andrewor14/ui-refactor
914b8ff [Tathagata Das] Moved utils functions to UIUtils.
548c98c [Andrew Or] Wide refactoring of WebUI, UITab, and UIPage (see commit message)
6de06b0 [Tathagata Das] Merge remote-tracking branch 'apache/master' into streaming-web-ui
ee6543f [Tathagata Das] Minor changes based on Andrew's comments.
fa760fe [Tathagata Das] Fixed long line.
1c0bcef [Tathagata Das] Refactored streaming UI into two files.
1af239b [Tathagata Das] Changed streaming UI to attach itself as a tab with the Spark UI.
827e81a [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui
168fe86 [Tathagata Das] Merge pull request #2 from andrewor14/ui-refactor
3e986f8 [Tathagata Das] Merge remote-tracking branch 'apache/master' into streaming-web-ui
c78c92d [Andrew Or] Remove outdated comment
8f7323b [Andrew Or] End of file new lines, indentation, and imports (minor)
0d61ee8 [Andrew Or] Merge branch 'streaming-web-ui' of github.com:tdas/spark into ui-refactor
9a48fa1 [Andrew Or] Allow adding tabs to SparkUI dynamically + add example
61358e3 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-web-ui
53be2c5 [Tathagata Das] Minor style updates.
ed25dfc [Andrew Or] Generalize SparkUI header to display tabs dynamically
a37ad4f [Andrew Or] Comments, imports and formatting (minor)
cd000b0 [Andrew Or] Merge github.com:apache/spark into ui-refactor
7d57444 [Andrew Or] Refactoring the UI interface to add flexibility
aef4dd5 [Tathagata Das] Added Apache licenses.
db27bad [Tathagata Das] Added last batch processing time to StreamingUI.
4d86e98 [Tathagata Das] Added basic stats to the StreamingUI and refactored the UI to a Page to make it easier to transition to using SparkUI later.
93f1c69 [Tathagata Das] Added network receiver information to the Streaming UI.
56cc7fb [Tathagata Das] First cut implementation of Streaming UI.
2014-04-11 23:33:49 -07:00
Xiangrui Meng 7038b00be9 [FIX] make coalesce test deterministic in RDDSuite
Make coalesce test deterministic by setting pre-defined seeds. (Saw random failures in other PRs.)

Author: Xiangrui Meng <meng@databricks.com>

Closes #387 from mengxr/fix-random and squashes the following commits:

59bc16f [Xiangrui Meng] make coalesce test deterministic in RDDSuite
2014-04-11 19:41:40 -07:00
Thomas Graves 446bb3417a SPARK-1417: Spark on Yarn - spark UI link from resourcemanager is broken
Author: Thomas Graves <tgraves@apache.org>

Closes #344 from tgravescs/SPARK-1417 and squashes the following commits:

c450b5f [Thomas Graves] fix test
e1c1d7e [Thomas Graves] add missing $ to appUIAddress
e982ddb [Thomas Graves] use appUIHostPort in appUIAddress
0803ec2 [Thomas Graves] Review comment updates - remove extra newline, simplify assert in test
658a8ec [Thomas Graves] Add a appUIHostPort routine
0614208 [Thomas Graves] Fix test
2a6b1b7 [Thomas Graves] SPARK-1417: Spark on Yarn - spark UI link from resourcemanager is broken
2014-04-11 13:17:48 +05:30
Sundeep Narravula 2c557837b4 SPARK-1202 - Add a "cancel" button in the UI for stages
Author: Sundeep Narravula <sundeepn@superduel.local>
Author: Sundeep Narravula <sundeepn@dhcpx-204-110.corp.yahoo.com>

Closes #246 from sundeepn/uikilljob and squashes the following commits:

5fdd0e2 [Sundeep Narravula] Fix test string
f6fdff1 [Sundeep Narravula] Format fix; reduced line size to less than 100 chars
d1daeb9 [Sundeep Narravula] Incorporating review comments.
8d97923 [Sundeep Narravula] Ability to kill jobs thru the UI. This behavior can be turned on be settings the following variable: spark.ui.killEnabled=true (default=false) Adding DAGScheduler event StageCancelled and corresponding handlers. Added cancellation reason to handlers.
2014-04-10 17:10:11 -07:00
Sandeep 930b70f052 Remove Unnecessary Whitespace's
stack these together in a commit else they show up chunk by chunk in different commits.

Author: Sandeep <sandeep@techaddict.me>

Closes #380 from techaddict/white_space and squashes the following commits:

b58f294 [Sandeep] Remove Unnecessary Whitespace's
2014-04-10 15:04:13 -07:00
Andrew Or 79820fe825 [SPARK-1276] Add a HistoryServer to render persisted UI
The new feature of event logging, introduced in #42, allows the user to persist the details of his/her Spark application to storage, and later replay these events to reconstruct an after-the-fact SparkUI.
Currently, however, a persisted UI can only be rendered through the standalone Master. This greatly limits the use case of this new feature as many people also run Spark on Yarn / Mesos.

This PR introduces a new entity called the HistoryServer, which, given a log directory, keeps track of all completed applications independently of a Spark Master. Unlike Master, the HistoryServer needs not be running while the application is still running. It is relatively light-weight in that it only maintains static information of applications and performs no scheduling.

To quickly test it out, generate event logs with ```spark.eventLog.enabled=true``` and run ```sbin/start-history-server.sh <log-dir-path>```. Your HistoryServer awaits on port 18080.

Comments and feedback are most welcome.

---

A few other changes introduced in this PR include refactoring the WebUI interface, which is beginning to have a lot of duplicate code now that we have added more functionality to it. Two new SparkListenerEvents have been introduced (SparkListenerApplicationStart/End) to keep track of application name and start/finish times. This PR also clarifies the semantics of the ReplayListenerBus introduced in #42.

A potential TODO in the future (not part of this PR) is to render live applications in addition to just completed applications. This is useful when applications fail, a condition that our current HistoryServer does not handle unless the user manually signals application completion (by creating the APPLICATION_COMPLETION file). Handling live applications becomes significantly more challenging, however, because it is now necessary to render the same SparkUI multiple times. To avoid reading the entire log every time, which is inefficient, we must handle reading the log from where we previously left off, but this becomes fairly complicated because we must deal with the arbitrary behavior of each input stream.

Author: Andrew Or <andrewor14@gmail.com>

Closes #204 from andrewor14/master and squashes the following commits:

7b7234c [Andrew Or] Finished -> Completed
b158d98 [Andrew Or] Address Patrick's comments
69d1b41 [Andrew Or] Do not block on posting SparkListenerApplicationEnd
19d5dd0 [Andrew Or] Merge github.com:apache/spark
f7f5bf0 [Andrew Or] Make history server's web UI port a Spark configuration
2dfb494 [Andrew Or] Decouple checking for application completion from replaying
d02dbaa [Andrew Or] Expose Spark version and include it in event logs
2282300 [Andrew Or] Add documentation for the HistoryServer
567474a [Andrew Or] Merge github.com:apache/spark
6edf052 [Andrew Or] Merge github.com:apache/spark
19e1fb4 [Andrew Or] Address Thomas' comments
248cb3d [Andrew Or] Limit number of live applications + add configurability
a3598de [Andrew Or] Do not close file system with ReplayBus + fix bind address
bc46fc8 [Andrew Or] Merge github.com:apache/spark
e2f4ff9 [Andrew Or] Merge github.com:apache/spark
050419e [Andrew Or] Merge github.com:apache/spark
81b568b [Andrew Or] Fix strange error messages...
0670743 [Andrew Or] Decouple page rendering from loading files from disk
1b2f391 [Andrew Or] Minor changes
a9eae7e [Andrew Or] Merge branch 'master' of github.com:apache/spark
d5154da [Andrew Or] Styling and comments
5dbfbb4 [Andrew Or] Merge branch 'master' of github.com:apache/spark
60bc6d5 [Andrew Or] First complete implementation of HistoryServer (only for finished apps)
7584418 [Andrew Or] Report application start/end times to HistoryServer
8aac163 [Andrew Or] Add basic application table
c086bd5 [Andrew Or] Add HistoryServer and scripts ++ Refactor WebUI interface
2014-04-10 10:39:34 -07:00
Patrick Wendell e6d4a74d2d Revert "SPARK-729: Closures not always serialized at capture time"
This reverts commit 8ca3b2bc90.
2014-04-10 02:10:40 -07:00
William Benton 8ca3b2bc90 SPARK-729: Closures not always serialized at capture time
[SPARK-729](https://spark-project.atlassian.net/browse/SPARK-729) concerns when free variables in closure arguments to transformations are captured.  Currently, it is possible for closures to get the environment in which they are serialized (not the environment in which they are created).  There are a few possible approaches to solving this problem and this PR will discuss some of them.  The approach I took has the advantage of being simple, obviously correct, and minimally-invasive, but it preserves something that has been bothering me about Spark's closure handling, so I'd like to discuss an alternative and get some feedback on whether or not it is worth pursuing.

## What I did

The basic approach I took depends on the work I did for #143, and so this PR is based atop that.  Specifically: #143 modifies `ClosureCleaner.clean` to preemptively determine whether or not closures are serializable immediately upon closure cleaning (rather than waiting for an job involving that closure to be scheduled).  Thus non-serializable closure exceptions will be triggered by the line defining the closure rather than triggered where the closure is used.

Since the easiest way to determine whether or not a closure is serializable is to attempt to serialize it, the code in #143 is creating a serialized closure as part of `ClosureCleaner.clean`.  `clean` currently modifies its argument, but the method in `SparkContext` that wraps it to return a value (a reference to the modified-in-place argument).  This branch modifies `ClosureCleaner.clean` so that it returns a value:  if it is cleaning a serializable closure, it returns the result of deserializing its serialized argument; therefore it is returning a closure with an environment captured at cleaning time.  `SparkContext.clean` then returns the result of `ClosureCleaner.clean`, rather than a reference to its modified-in-place argument.

I've added tests for this behavior (777a1bc).  The pull request as it stands, given the changes in #143, is nearly trivial.  There is some overhead from deserializing the closure, but it is minimal and the benefit of obvious operational correctness (vs. a more sophisticated but harder-to-validate transformation in `ClosureCleaner`) seems pretty important.  I think this is a fine way to solve this problem, but it's not perfect.

## What we might want to do

The thing that has been bothering me about Spark's handling of closures is that it seems like we should be able to statically ensure that cleaning and serialization happen exactly once for a given closure.  If we serialize a closure in order to determine whether or not it is serializable, we should be able to hang on to the generated byte buffer and use it instead of re-serializing the closure later.  By replacing closures with instances of a sum type that encodes whether or not a closure has been cleaned or serialized, we could handle clean, to-be-cleaned, and serialized closures separately with case matches.  Here's a somewhat-concrete sketch (taken from my git stash) of what this might look like:

```scala
package org.apache.spark.util

import java.nio.ByteBuffer
import scala.reflect.ClassManifest

sealed abstract class ClosureBox[T] { def func: T }
final case class RawClosure[T](func: T) extends ClosureBox[T] {}
final case class CleanedClosure[T](func: T) extends ClosureBox[T] {}
final case class SerializedClosure[T](func: T, bytebuf: ByteBuffer) extends ClosureBox[T] {}

object ClosureBoxImplicits {
  implicit def closureBoxFromFunc[T <: AnyRef](fun: T) = new RawClosure[T](fun)
}
```

With these types declared, we'd be able to change `ClosureCleaner.clean` to take a `ClosureBox[T=>U]` (possibly generated by implicit conversion) and return a `ClosureBox[T=>U]` (either a `CleanedClosure[T=>U]` or a `SerializedClosure[T=>U]`, depending on whether or not serializability-checking was enabled) instead of a `T=>U`.  A case match could thus short-circuit cleaning or serializing closures that had already been cleaned or serialized (both in `ClosureCleaner` and in the closure serializer).  Cleaned-and-serialized closures would be represented by a boxed tuple of the original closure and a serialized copy (complete with an environment quiesced at transformation time).  Additional implicit conversions could convert from `ClosureBox` instances to the underlying function type where appropriate.  Tracking this sort of state in the type system seems like the right thing to do to me.

### Why we might not want to do that

_It's pretty invasive._  Every function type used by every `RDD` subclass would have to change to reflect that they expected a `ClosureBox[T=>U]` instead of a `T=>U`.  This obscures what's going on and is not a little ugly.  Although I really like the idea of using the type system to enforce the clean-or-serialize once discipline, it might not be worth adding another layer of types (even if we could hide some of the extra boilerplate with judicious application of implicit conversions).

_It statically guarantees a property whose absence is unlikely to cause any serious problems as it stands._  It appears that all closures are currently dynamically cleaned once and it's not obvious that repeated closure-cleaning is likely to be a problem in the future.  Furthermore, serializing closures is relatively cheap, so doing it once to check for serialization and once again to actually ship them across the wire doesn't seem like a big deal.

Taken together, these seem like a high price to pay for statically guaranteeing that closures are operated upon only once.

## Other possibilities

I felt like the serialize-and-deserialize approach was best due to its obvious simplicity.  But it would be possible to do a more sophisticated transformation within `ClosureCleaner.clean`.  It might also be possible for `clean` to modify its argument in a way so that whether or not a given closure had been cleaned would be apparent upon inspection; this would buy us some of the operational benefits of the `ClosureBox` approach but not the static cleanliness.

I'm interested in any feedback or discussion on whether or not the problems with the type-based approach indeed outweigh the advantage, as well as of approaches to this issue and to closure handling in general.

Author: William Benton <willb@redhat.com>

Closes #189 from willb/spark-729 and squashes the following commits:

f4cafa0 [William Benton] Stylistic changes and cleanups
b3d9c86 [William Benton] Fixed style issues in tests
9b56ce0 [William Benton] Added array-element capture test
97e9d91 [William Benton] Split closure-serializability failure tests
12ef6e3 [William Benton] Skip proactive closure capture for runJob
8ee3ee7 [William Benton] Predictable closure environment capture
12c63a7 [William Benton] Added tests for variable capture in closures
d6e8dd6 [William Benton] Don't check serializability of DStream transforms.
4ecf841 [William Benton] Make proactive serializability checking optional.
d8df3db [William Benton] Adds proactive closure-serializablilty checking
21b4b06 [William Benton] Test cases for SPARK-897.
d5947b3 [William Benton] Ensure assertions in Graph.apply are asserted.
2014-04-09 18:56:27 -07:00
Kan Zhang eb5f2b6423 SPARK-1407 drain event queue before stopping event logger
Author: Kan Zhang <kzhang@apache.org>

Closes #366 from kanzhang/SPARK-1407 and squashes the following commits:

cd0629f [Kan Zhang] code refactoring and adding test
b073ee6 [Kan Zhang] SPARK-1407 drain event queue before stopping event logger
2014-04-09 15:25:29 -07:00
Holden Karau fa0524fd02 Spark-939: allow user jars to take precedence over spark jars
I still need to do a small bit of re-factoring [mostly the one Java file I'll switch it back to a Scala file and use it in both the close loaders], but comments on other things I should do would be great.

Author: Holden Karau <holden@pigscanfly.ca>

Closes #217 from holdenk/spark-939-allow-user-jars-to-take-precedence-over-spark-jars and squashes the following commits:

cf0cac9 [Holden Karau] Fix the executorclassloader
1955232 [Holden Karau] Fix long line in TestUtils
8f89965 [Holden Karau] Fix tests for new class name
7546549 [Holden Karau] CR feedback, merge some of the testutils methods down, rename the classloader
644719f [Holden Karau] User the class generator for the repl class loader tests too
f0b7114 [Holden Karau] Fix the core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala tests
204b199 [Holden Karau] Fix the generated classes
9f68f10 [Holden Karau] Start rewriting the ExecutorURLClassLoaderSuite to not use the hard coded classes
858aba2 [Holden Karau] Remove a bunch of test junk
261aaee [Holden Karau] simplify executorurlclassloader a bit
7a7bf5f [Holden Karau] CR feedback
d4ae848 [Holden Karau] rewrite component into scala
aa95083 [Holden Karau] CR feedback
7752594 [Holden Karau] re-add https comment
a0ef85a [Holden Karau] Fix style issues
125ea7f [Holden Karau] Easier to just remove those files, we don't need them
bb8d179 [Holden Karau] Fix issues with the repl class loader
241b03d [Holden Karau] fix my rat excludes
a343350 [Holden Karau] Update rat-excludes and remove a useless file
d90d217 [Holden Karau] Fix fall back with custom class loader and add a test for it
4919bf9 [Holden Karau] Fix parent calling class loader issue
8a67302 [Holden Karau] Test are good
9e2d236 [Holden Karau] It works comrade
691ee00 [Holden Karau] It works ish
dc4fe44 [Holden Karau] Does not depend on being in my home directory
47046ff [Holden Karau] Remove bad import'
22d83cb [Holden Karau] Add a test suite for the executor url class loader suite
7ef4628 [Holden Karau] Clean up
792d961 [Holden Karau] Almost works
16aecd1 [Holden Karau] Doesn't quite work
8d2241e [Holden Karau] Adda FakeClass for testing ClassLoader precedence options
648b559 [Holden Karau] Both class loaders compile. Now for testing
e1d9f71 [Holden Karau] One loader workers.
2014-04-08 22:30:03 -07:00
Holden Karau ce8ec54561 Spark 1271: Co-Group and Group-By should pass Iterable[X]
Author: Holden Karau <holden@pigscanfly.ca>

Closes #242 from holdenk/spark-1320-cogroupandgroupshouldpassiterator and squashes the following commits:

f289536 [Holden Karau] Fix bad merge, should have been Iterable rather than Iterator
77048f8 [Holden Karau] Fix merge up to master
d3fe909 [Holden Karau] use toSeq instead
7a092a3 [Holden Karau] switch resultitr to resultiterable
eb06216 [Holden Karau] maybe I should have had a coffee first. use correct import for guava iterables
c5075aa [Holden Karau] If guava 14 had iterables
2d06e10 [Holden Karau] Fix Java 8 cogroup tests for the new API
11e730c [Holden Karau] Fix streaming tests
66b583d [Holden Karau] Fix the core test suite to compile
4ed579b [Holden Karau] Refactor from iterator to iterable
d052c07 [Holden Karau] Python tests now pass with iterator pandas
3bcd81d [Holden Karau] Revert "Try and make pickling list iterators work"
cd1e81c [Holden Karau] Try and make pickling list iterators work
c60233a [Holden Karau] Start investigating moving to iterators for python API like the Java/Scala one. tl;dr: We will have to write our own iterator since the default one doesn't pickle well
88a5cef [Holden Karau] Fix cogroup test in JavaAPISuite for streaming
a5ee714 [Holden Karau] oops, was checking wrong iterator
e687f21 [Holden Karau] Fix groupbykey test in JavaAPISuite of streaming
ec8cc3e [Holden Karau] Fix test issues\!
4b0eeb9 [Holden Karau] Switch cast in PairDStreamFunctions
fa395c9 [Holden Karau] Revert "Add a join based on the problem in SVD"
ec99e32 [Holden Karau] Revert "Revert this but for now put things in list pandas"
b692868 [Holden Karau] Revert
7e533f7 [Holden Karau] Fix the bug
8a5153a [Holden Karau] Revert me, but we have some stuff to debug
b4e86a9 [Holden Karau] Add a join based on the problem in SVD
c4510e2 [Holden Karau] Revert this but for now put things in list pandas
b4e0b1d [Holden Karau] Fix style issues
71e8b9f [Holden Karau] I really need to stop calling size on iterators, it is the path of sadness.
b1ae51a [Holden Karau] Fix some of the types in the streaming JavaAPI suite. Probably still needs more work
37888ec [Holden Karau] core/tests now pass
249abde [Holden Karau] org.apache.spark.rdd.PairRDDFunctionsSuite passes
6698186 [Holden Karau] Revert "I think this might be a bad rabbit hole. Started work to make CoGroupedRDD use iterator and then went crazy"
fe992fe [Holden Karau] hmmm try and fix up basic operation suite
172705c [Holden Karau] Fix Java API suite
caafa63 [Holden Karau] I think this might be a bad rabbit hole. Started work to make CoGroupedRDD use iterator and then went crazy
88b3329 [Holden Karau] Fix groupbykey to actually give back an iterator
4991af6 [Holden Karau] Fix some tests
be50246 [Holden Karau] Calling size on an iterator is not so good if we want to use it after
687ffbc [Holden Karau] This is the it compiles point of replacing Seq with Iterator and JList with JIterator in the groupby and cogroup signatures
2014-04-08 18:15:59 -07:00
Kay Ousterhout fac6085cd7 [SPARK-1397] Notify SparkListeners when stages fail or are cancelled.
[I wanted to post this for folks to comment but it depends on (and thus includes the changes in) a currently outstanding PR, #305.  You can look at just the second commit: 93f08baf73 to see just the changes relevant to this PR]

Previously, when stages fail or get cancelled, the SparkListener is only notified
indirectly through the SparkListenerJobEnd, where we sometimes pass in a single
stage that failed.  This worked before job cancellation, because jobs would only fail
due to a single stage failure.  However, with job cancellation, multiple running stages
can fail when a job gets cancelled.  Right now, this is not handled correctly, which
results in stages that get stuck in the “Running Stages” window in the UI even
though they’re dead.

This PR changes the SparkListenerStageCompleted event to a SparkListenerStageEnded
event, and uses this event to tell SparkListeners when stages fail in addition to when
they complete successfully.  This change is NOT publicly backward compatible for two
reasons.  First, it changes the SparkListener interface.  We could alternately add a new event,
SparkListenerStageFailed, and keep the existing SparkListenerStageCompleted.  However,
this is less consistent with the listener events for tasks / jobs ending, and will result in some
code duplication for listeners (because failed and completed stages are handled in similar
ways).  Note that I haven’t finished updating the JSON code to correctly handle the new event
because I’m waiting for feedback on whether this is a good or bad idea (hence the “WIP”).

It is also not backwards compatible because it changes the publicly visible JobWaiter.jobFailed()
method to no longer include a stage that caused the failure.  I think this change should definitely
stay, because with cancellation (as described above), a failure isn’t necessarily caused by a
single stage.

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #309 from kayousterhout/stage_cancellation and squashes the following commits:

5533ecd [Kay Ousterhout] Fixes in response to Mark's review
320c7c7 [Kay Ousterhout] Notify SparkListeners when stages fail or are cancelled.
2014-04-08 14:42:02 -07:00
Kay Ousterhout 6dc5f5849c [SPARK-1396] Properly cleanup DAGScheduler on job cancellation.
Previously, when jobs were cancelled, not all of the state in the
DAGScheduler was cleaned up, leading to a slow memory leak in the
DAGScheduler.  As we expose easier ways to cancel jobs, it's more
important to fix these issues.

This commit also fixes a second and less serious problem, which is that
previously, when a stage failed, not all of the appropriate stages
were cancelled.  See the "failure of stage used by two jobs" test
for an example of this.  This just meant that extra work was done, and is
not a correctness problem.

This commit adds 3 tests.  “run shuffle with map stage failure” is
a new test to more thoroughly test this functionality, and passes on
both the old and new versions of the code.  “trivial job
cancellation” fails on the old code because all state wasn’t cleaned
up correctly when jobs were cancelled (we didn’t remove the job from
resultStageToJob).  “failure of stage used by two jobs” fails on the
old code because taskScheduler.cancelTasks wasn’t called for one of
the stages (see test comments).

This should be checked in before #246, which makes it easier to
cancel stages / jobs.

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #305 from kayousterhout/incremental_abort_fix and squashes the following commits:

f33d844 [Kay Ousterhout] Mark review comments
9217080 [Kay Ousterhout] Properly cleanup DAGScheduler on job cancellation.
2014-04-08 01:03:33 -07:00
Tathagata Das 11eabbe125 [SPARK-1103] Automatic garbage collection of RDD, shuffle and broadcast data
This PR allows Spark to automatically cleanup metadata and data related to persisted RDDs, shuffles and broadcast variables when the corresponding RDDs, shuffles and broadcast variables fall out of scope from the driver program. This is still a work in progress as broadcast cleanup has not been implemented.

**Implementation Details**
A new class `ContextCleaner` is responsible cleaning all the state. It is instantiated as part of a `SparkContext`. RDD and ShuffleDependency classes have overridden `finalize()` function that gets called whenever their instances go out of scope. The `finalize()` function enqueues the object’s identifier (i.e. RDD ID, shuffle ID, etc.) with the `ContextCleaner`, which is a very short and cheap operation and should not significantly affect the garbage collection mechanism. The `ContextCleaner`, on a different thread, performs the cleanup, whose details are given below.

*RDD cleanup:*
`ContextCleaner` calls `RDD.unpersist()` is used to cleanup persisted RDDs. Regarding metadata, the DAGScheduler automatically cleans up all metadata related to a RDD after all jobs have completed. Only the `SparkContext.persistentRDDs` keeps strong references to persisted RDDs. The `TimeStampedHashMap` used for that has been replaced by `TimeStampedWeakValueHashMap` that keeps only weak references to the RDDs, allowing them to be garbage collected.

*Shuffle cleanup:*
New BlockManager message `RemoveShuffle(<shuffle ID>)` asks the `BlockManagerMaster` and currently active `BlockManager`s to delete all the disk blocks related to the shuffle ID. `ContextCleaner` cleans up shuffle data using this message and also cleans up the metadata in the `MapOutputTracker` of the driver. The `MapOutputTracker` at the workers, that caches the shuffle metadata, maintains a `BoundedHashMap` to limit the shuffle information it caches. Refetching the shuffle information from the driver is not too costly.

*Broadcast cleanup:*
To be done. [This PR](https://github.com/apache/incubator-spark/pull/543/) adds mechanism for explicit cleanup of broadcast variables. `Broadcast.finalize()` will enqueue its own ID with ContextCleaner and the PRs mechanism will be used to unpersist the Broadcast data.

*Other cleanup:*
`ShuffleMapTask` and `ResultTask` caches tasks and used TTL based cleanup (using `TimeStampedHashMap`), so nothing got cleaned up if TTL was not set. Instead, they now use `BoundedHashMap` to keep a limited number of map output information. Cost of repopulating the cache if necessary is very small.

**Current state of implementation**
Implemented RDD and shuffle cleanup. Things left to be done are.
- Cleaning up for broadcast variable still to be done.
- Automatic cleaning up keys with empty weak refs as values in `TimeStampedWeakValueHashMap`

Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Andrew Or <andrewor14@gmail.com>
Author: Roman Pastukhov <ignatich@mail.ru>

Closes #126 from tdas/state-cleanup and squashes the following commits:

61b8d6e [Tathagata Das] Fixed issue with Tachyon + new BlockManager methods.
f489fdc [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup
d25a86e [Tathagata Das] Fixed stupid typo.
cff023c [Tathagata Das] Fixed issues based on Andrew's comments.
4d05314 [Tathagata Das] Scala style fix.
2b95b5e [Tathagata Das] Added more documentation on Broadcast implementations, specially which blocks are told about to the driver. Also, fixed Broadcast API to hide destroy functionality.
41c9ece [Tathagata Das] Added more unit tests for BlockManager, DiskBlockManager, and ContextCleaner.
6222697 [Tathagata Das] Fixed bug and adding unit test for removeBroadcast in BlockManagerSuite.
104a89a [Tathagata Das] Fixed failing BroadcastSuite unit tests by introducing blocking for removeShuffle and removeBroadcast in BlockManager*
a430f06 [Tathagata Das] Fixed compilation errors.
b27f8e8 [Tathagata Das] Merge pull request #3 from andrewor14/cleanup
cd72d19 [Andrew Or] Make automatic cleanup configurable (not documented)
ada45f0 [Andrew Or] Merge branch 'state-cleanup' of github.com:tdas/spark into cleanup
a2cc8bc [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup
c5b1d98 [Andrew Or] Address Patrick's comments
a6460d4 [Andrew Or] Merge github.com:apache/spark into cleanup
762a4d8 [Tathagata Das] Merge pull request #1 from andrewor14/cleanup
f0aabb1 [Andrew Or] Correct semantics for TimeStampedWeakValueHashMap + add tests
5016375 [Andrew Or] Address TD's comments
7ed72fb [Andrew Or] Fix style test fail + remove verbose test message regarding broadcast
634a097 [Andrew Or] Merge branch 'state-cleanup' of github.com:tdas/spark into cleanup
7edbc98 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into state-cleanup
8557c12 [Andrew Or] Merge github.com:apache/spark into cleanup
e442246 [Andrew Or] Merge github.com:apache/spark into cleanup
88904a3 [Andrew Or] Make TimeStampedWeakValueHashMap a wrapper of TimeStampedHashMap
fbfeec8 [Andrew Or] Add functionality to query executors for their local BlockStatuses
34f436f [Andrew Or] Generalize BroadcastBlockId to remove BroadcastHelperBlockId
0d17060 [Andrew Or] Import, comments, and style fixes (minor)
c92e4d9 [Andrew Or] Merge github.com:apache/spark into cleanup
f201a8d [Andrew Or] Test broadcast cleanup in ContextCleanerSuite + remove BoundedHashMap
e95479c [Andrew Or] Add tests for unpersisting broadcast
544ac86 [Andrew Or] Clean up broadcast blocks through BlockManager*
d0edef3 [Andrew Or] Add framework for broadcast cleanup
ba52e00 [Andrew Or] Refactor broadcast classes
c7ccef1 [Andrew Or] Merge branch 'bc-unpersist-merge' of github.com:ignatich/incubator-spark into cleanup
6c9dcf6 [Tathagata Das] Added missing Apache license
d2f8b97 [Tathagata Das] Removed duplicate unpersistRDD.
a007307 [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup
620eca3 [Tathagata Das] Changes based on PR comments.
f2881fd [Tathagata Das] Changed ContextCleaner to use ReferenceQueue instead of finalizer
e1fba5f [Tathagata Das] Style fix
892b952 [Tathagata Das] Removed use of BoundedHashMap, and made BlockManagerSlaveActor cleanup shuffle metadata in MapOutputTrackerWorker.
a7260d3 [Tathagata Das] Added try-catch in context cleaner and null value cleaning in TimeStampedWeakValueHashMap.
e61daa0 [Tathagata Das] Modifications based on the comments on PR 126.
ae9da88 [Tathagata Das] Removed unncessary TimeStampedHashMap from DAGScheduler, added try-catches in finalize() methods, and replaced ArrayBlockingQueue to LinkedBlockingQueue to avoid blocking in Java's finalizing thread.
cb0a5a6 [Tathagata Das] Fixed docs and styles.
a24fefc [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup
8512612 [Tathagata Das] Changed TimeStampedHashMap to use WrappedJavaHashMap.
e427a9e [Tathagata Das] Added ContextCleaner to automatically clean RDDs and shuffles when they fall out of scope. Also replaced TimeStampedHashMap to BoundedHashMaps and TimeStampedWeakValueHashMap for the necessary hashmap behavior.
80dd977 [Roman Pastukhov] Fix for Broadcast unpersist patch.
1e752f1 [Roman Pastukhov] Added unpersist method to Broadcast.
2014-04-07 23:40:36 -07:00
Aaron Davidson 0307db0f55 SPARK-1099: Introduce local[*] mode to infer number of cores
This is the default mode for running spark-shell and pyspark, intended to allow users running spark for the first time to see the performance benefits of using multiple cores, while not breaking backwards compatibility for users who use "local" mode and expect exactly 1 core.

Author: Aaron Davidson <aaron@databricks.com>

Closes #182 from aarondav/110 and squashes the following commits:

a88294c [Aaron Davidson] Rebased changes for new spark-shell
a9f393e [Aaron Davidson] SPARK-1099: Introduce local[*] mode to infer number of cores
2014-04-07 13:06:30 -07:00
Evan Chan 1440154c27 SPARK-1154: Clean up app folders in worker nodes
This is a fix for [SPARK-1154](https://issues.apache.org/jira/browse/SPARK-1154).   The issue is that worker nodes fill up with a huge number of app-* folders after some time.  This change adds a periodic cleanup task which asynchronously deletes app directories older than a configurable TTL.

Two new configuration parameters have been introduced:
  spark.worker.cleanup_interval
  spark.worker.app_data_ttl

This change does not include moving the downloads of application jars to a location outside of the work directory.  We will address that if we have time, but that potentially involves caching so it will come either as part of this PR or a separate PR.

Author: Evan Chan <ev@ooyala.com>
Author: Kelvin Chu <kelvinkwchu@yahoo.com>

Closes #288 from velvia/SPARK-1154-cleanup-app-folders and squashes the following commits:

0689995 [Evan Chan] CR from @aarondav - move config, clarify for standalone mode
9f10d96 [Evan Chan] CR from @pwendell - rename configs and add cleanup.enabled
f2f6027 [Evan Chan] CR from @andrewor14
553d8c2 [Kelvin Chu] change the variable name to currentTimeMillis since it actually tracks in seconds
8dc9cb5 [Kelvin Chu] Fixed a bug in Utils.findOldFiles() after merge.
cb52f2b [Kelvin Chu] Change the name of findOldestFiles() to findOldFiles()
72f7d2d [Kelvin Chu] Fix a bug of Utils.findOldestFiles(). file.lastModified is returned in milliseconds.
ad99955 [Kelvin Chu] Add unit test for Utils.findOldestFiles()
dc1a311 [Evan Chan] Don't recompute current time with every new file
e3c408e [Evan Chan] Document the two new settings
b92752b [Evan Chan] SPARK-1154: Add a periodic task to clean up app directories
2014-04-06 19:21:40 -07:00
Egor Pakhomov e258e5040f [SPARK-1259] Make RDD locally iterable
Author: Egor Pakhomov <pahomov.egor@gmail.com>

Closes #156 from epahomov/SPARK-1259 and squashes the following commits:

8ec8f24 [Egor Pakhomov] Make to local iterator shorter
34aa300 [Egor Pakhomov] Fix toLocalIterator docs
08363ef [Egor Pakhomov] SPARK-1259 from toLocallyIterable to toLocalIterator
6a994eb [Egor Pakhomov] SPARK-1259 Make RDD locally iterable
8be3dcf [Egor Pakhomov] SPARK-1259 Make RDD locally iterable
33ecb17 [Egor Pakhomov] SPARK-1259 Make RDD locally iterable
2014-04-06 16:43:01 -07:00
Haoyuan Li b50ddfde03 SPARK-1305: Support persisting RDD's directly to Tachyon
Move the PR#468 of apache-incubator-spark to the apache-spark
"Adding an option to persist Spark RDD blocks into Tachyon."

Author: Haoyuan Li <haoyuan@cs.berkeley.edu>
Author: RongGu <gurongwalker@gmail.com>

Closes #158 from RongGu/master and squashes the following commits:

72b7768 [Haoyuan Li] merge master
9f7fa1b [Haoyuan Li] fix code style
ae7834b [Haoyuan Li] minor cleanup
a8b3ec6 [Haoyuan Li] merge master branch
e0f4891 [Haoyuan Li] better check offheap.
55b5918 [RongGu] address matei's comment on the replication of offHeap storagelevel
7cd4600 [RongGu] remove some logic code for tachyonstore's replication
51149e7 [RongGu] address aaron's comment on returning value of the remove() function in tachyonstore
8adfcfa [RongGu] address arron's comment on inTachyonSize
120e48a [RongGu] changed the root-level dir name in Tachyon
5cc041c [Haoyuan Li] address aaron's comments
9b97935 [Haoyuan Li] address aaron's comments
d9a6438 [Haoyuan Li] fix for pspark
77d2703 [Haoyuan Li] change python api.git status
3dcace4 [Haoyuan Li] address matei's comments
91fa09d [Haoyuan Li] address patrick's comments
589eafe [Haoyuan Li] use TRY_CACHE instead of MUST_CACHE
64348b2 [Haoyuan Li] update conf docs.
ed73e19 [Haoyuan Li] Merge branch 'master' of github.com:RongGu/spark-1
619a9a8 [RongGu] set number of directories in TachyonStore back to 64; added a TODO tag for duplicated code from the DiskStore
be79d77 [RongGu] find a way to clean up some unnecessay metods and classed to make the code simpler
49cc724 [Haoyuan Li] update docs with off_headp option
4572f9f [RongGu] reserving the old apply function API of StorageLevel
04301d3 [RongGu] rename StorageLevel.TACHYON to Storage.OFF_HEAP
c9aeabf [RongGu] rename the StorgeLevel.TACHYON as StorageLevel.OFF_HEAP
76805aa [RongGu] unifies the config properties name prefix; add the configs into docs/configuration.md
e700d9c [RongGu] add the SparkTachyonHdfsLR example and some comments
fd84156 [RongGu] use randomUUID to generate sparkapp directory name on tachyon;minor code style fix
939e467 [Haoyuan Li] 0.4.1-thrift from maven central
86a2eab [Haoyuan Li] tachyon 0.4.1-thrift is in the staging repo. but jenkins failed to download it. temporarily revert it back to 0.4.1
16c5798 [RongGu] make the dependency on tachyon as tachyon-0.4.1-thrift
eacb2e8 [RongGu] Merge branch 'master' of https://github.com/RongGu/spark-1
bbeb4de [RongGu] fix the JsonProtocolSuite test failure problem
6adb58f [RongGu] Merge branch 'master' of https://github.com/RongGu/spark-1
d827250 [RongGu] fix JsonProtocolSuie test failure
716e93b [Haoyuan Li] revert the version
ca14469 [Haoyuan Li] bump tachyon version to 0.4.1-thrift
2825a13 [RongGu] up-merging to the current master branch of the apache spark
6a22c1a [Haoyuan Li] fix scalastyle
8968b67 [Haoyuan Li] exclude more libraries from tachyon dependency to be the same as referencing tachyon-client.
77be7e8 [RongGu] address mateiz's comment about the temp folder name problem. The implementation followed mateiz's advice.
1dcadf9 [Haoyuan Li] typo
bf278fa [Haoyuan Li] fix python tests
e82909c [Haoyuan Li] minor cleanup
776a56c [Haoyuan Li] address patrick's and ali's comments from the previous PR
8859371 [Haoyuan Li] various minor fixes and clean up
e3ddbba [Haoyuan Li] add doc to use Tachyon cache mode.
fcaeab2 [Haoyuan Li] address Aaron's comment
e554b1e [Haoyuan Li] add python code
47304b3 [Haoyuan Li] make tachyonStore in BlockMananger lazy val; add more comments StorageLevels.
dc8ef24 [Haoyuan Li] add old storelevel constructor
e01a271 [Haoyuan Li] update tachyon 0.4.1
8011a96 [RongGu] fix a brought-in mistake in StorageLevel
70ca182 [RongGu] a bit change in comment
556978b [RongGu] fix the scalastyle errors
791189b [RongGu] "Adding an option to persist Spark RDD blocks into Tachyon." move the PR#468 of apache-incubator-spark to the apache-spark
2014-04-04 20:38:20 -07:00
Patrick Wendell 5f3c1bb513 Add test utility for generating Jar files with compiled classes.
This was requested by a few different people and may be generally
useful, so I'd like to contribute this and not block on a different
PR for it to get in.

Author: Patrick Wendell <pwendell@gmail.com>

Closes #326 from pwendell/class-loader-test-utils and squashes the following commits:

ff3e88e [Patrick Wendell] Add test utility for generating Jar files with compiled classes.
2014-04-04 19:15:15 -07:00
Thomas Graves 198892fe8d [SPARK-1198] Allow pipes tasks to run in different sub-directories
This works as is on Linux/Mac/etc but doesn't cover working on Windows.  In here I use ln -sf for symlinks. Putting this up for comments on that. Do we want to create perhaps some classes for doing shell commands - Linux vs Windows.  Is there some other way we want to do this?   I assume we are still supporting jdk1.6?

Also should I update the Java API for pipes to allow this parameter?

Author: Thomas Graves <tgraves@apache.org>

Closes #128 from tgravescs/SPARK1198 and squashes the following commits:

abc1289 [Thomas Graves] remove extra tag in pom file
ba23fc0 [Thomas Graves] Add support for symlink on windows, remove commons-io usage
da4b221 [Thomas Graves] Merge branch 'master' of https://github.com/tgravescs/spark into SPARK1198
61be271 [Thomas Graves] Fix file name filter
6b783bd [Thomas Graves] style fixes
1ab49ca [Thomas Graves] Add support for running pipe tasks is separate directories
2014-04-04 17:16:31 -07:00
Patrick Wendell a02b535d5e Don't create SparkContext in JobProgressListenerSuite.
This reduces the time of the test from 11 seconds to 20 milliseconds.

Author: Patrick Wendell <pwendell@gmail.com>

Closes #324 from pwendell/job-test and squashes the following commits:

868d9eb [Patrick Wendell] Don't create SparkContext in JobProgressListenerSuite.
2014-04-04 14:46:32 -07:00
Xusen Yin f1fa617023 [SPARK-1133] Add whole text files reader in MLlib
Here is a pointer to the former [PR164](https://github.com/apache/spark/pull/164).

I add the pull request for the JIRA issue [SPARK-1133](https://spark-project.atlassian.net/browse/SPARK-1133), which brings a new files reader API in MLlib.

Author: Xusen Yin <yinxusen@gmail.com>

Closes #252 from yinxusen/whole-files-input and squashes the following commits:

7191be6 [Xusen Yin] refine comments
0af3faf [Xusen Yin] add JavaAPI test
01745ee [Xusen Yin] fix deletion error
cc97dca [Xusen Yin] move whole text file API to Spark core
d792cee [Xusen Yin] remove the typo character "+"
6bdf2c2 [Xusen Yin] test for small local file system block size
a1f1e7e [Xusen Yin] add two extra spaces
28cb0fe [Xusen Yin] add whole text files reader
2014-04-04 11:12:47 -07:00
Patrick Wendell ee6e9e7d86 SPARK-1337: Application web UI garbage collects newest stages
Simple fix...

Author: Patrick Wendell <pwendell@gmail.com>

Closes #320 from pwendell/stage-clean-up and squashes the following commits:

29be62e [Patrick Wendell] SPARK-1337: Application web UI garbage collects newest stages instead old ones
2014-04-03 22:13:56 -07:00
Andrew Or de8eefa804 [SPARK-1385] Use existing code for JSON de/serialization of BlockId
`BlockId.scala` offers a way to reconstruct a BlockId from a string through regex matching. `util/JsonProtocol.scala` duplicates this functionality by explicitly matching on the BlockId type.
With this PR, the de/serialization of BlockIds will go through the first (older) code path.

(Most of the line changes in this PR involve changing `==` to `===` in `JsonProtocolSuite.scala`)

Author: Andrew Or <andrewor14@gmail.com>

Closes #289 from andrewor14/blockid-json and squashes the following commits:

409d226 [Andrew Or] Simplify JSON de/serialization for BlockId
2014-04-02 10:43:09 -07:00
Kay Ousterhout 11973a7bda Renamed stageIdToActiveJob to jobIdToActiveJob.
This data structure was misused and, as a result, later renamed to an incorrect name.

This data structure seems to have gotten into this tangled state as a result of @henrydavidge using the stageID instead of the job Id to index into it and later @andrewor14 renaming the data structure to reflect this misunderstanding.

This patch renames it and removes an incorrect indexing into it.  The incorrect indexing into it meant that the code added by @henrydavidge to warn when a task size is too large (added here 57579934f0) was not always executed; this commit fixes that.

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #301 from kayousterhout/fixCancellation and squashes the following commits:

bd3d3a4 [Kay Ousterhout] Renamed stageIdToActiveJob to jobIdToActiveJob.
2014-04-02 10:35:52 -07:00
Patrick Wendell 841721e03c SPARK-1352: Improve robustness of spark-submit script
1. Better error messages when required arguments are missing.
2. Support for unit testing cases where presented arguments are invalid.
3. Bug fix: Only use environment varaibles when they are set (otherwise will cause NPE).
4. A verbose mode to aid debugging.
5. Visibility of several variables is set to private.
6. Deprecation warning for existing scripts.

Author: Patrick Wendell <pwendell@gmail.com>

Closes #271 from pwendell/spark-submit and squashes the following commits:

9146def [Patrick Wendell] SPARK-1352: Improve robustness of spark-submit script
2014-03-31 12:07:14 -07:00
Sandy Ryza 1617816090 SPARK-1126. spark-app preliminary
This is a starting version of the spark-app script for running compiled binaries against Spark.  It still needs tests and some polish.  The only testing I've done so far has been using it to launch jobs in yarn-standalone mode against a pseudo-distributed cluster.

This leaves out the changes required for launching python scripts.  I think it might be best to save those for another JIRA/PR (while keeping to the design so that they won't require backwards-incompatible changes).

Author: Sandy Ryza <sandy@cloudera.com>

Closes #86 from sryza/sandy-spark-1126 and squashes the following commits:

d428d85 [Sandy Ryza] Commenting, doc, and import fixes from Patrick's comments
e7315c6 [Sandy Ryza] Fix failing tests
34de899 [Sandy Ryza] Change --more-jars to --jars and fix docs
299ddca [Sandy Ryza] Fix scalastyle
a94c627 [Sandy Ryza] Add newline at end of SparkSubmit
04bc4e2 [Sandy Ryza] SPARK-1126. spark-submit script
2014-03-29 14:41:36 -07:00
Prashant Sharma 60abc25254 SPARK-1096, a space after comment start style checker.
Author: Prashant Sharma <prashant.s@imaginea.com>

Closes #124 from ScrapCodes/SPARK-1096/scalastyle-comment-check and squashes the following commits:

214135a [Prashant Sharma] Review feedback.
5eba88c [Prashant Sharma] Fixed style checks for ///+ comments.
e54b2f8 [Prashant Sharma] improved message, work around.
83e7144 [Prashant Sharma] removed dependency on scalastyle in plugin, since scalastyle sbt plugin already depends on the right version. Incase we update the plugin we will have to adjust our spark-style project to depend on right scalastyle version.
810a1d6 [Prashant Sharma] SPARK-1096, a space after comment style checker.
ba33193 [Prashant Sharma] scala style as a project
2014-03-28 00:21:49 -07:00
Petko Nikolov 6f986f0b87 [SPARK-1268] Adding XOR and AND-NOT operations to spark.util.collection.BitSet
Symmetric difference (xor) in particular is useful for computing some distance metrics (e.g. Hamming). Unit tests added.

Author: Petko Nikolov <nikolov@soundcloud.com>

Closes #172 from petko-nikolov/bitset-imprv and squashes the following commits:

451f28b [Petko Nikolov] fixed style mistakes
5beba18 [Petko Nikolov] rm outer loop in andNot test
0e61035 [Petko Nikolov] conform to spark style; rm redundant asserts; more unit tests added; use arraycopy instead of loop
d53cdb9 [Petko Nikolov] rm incidentally added space
4e1df43 [Petko Nikolov] adding xor and and-not to BitSet; unit tests added
2014-03-27 15:49:07 -07:00
witgo 8237df8060 Avoid Option while generating call site
This is an update on https://github.com/apache/spark/pull/180, which changes the solution from blacklisting "Option.scala" to avoiding the Option code path while generating the call path.

Also includes a unit test to prevent this issue in the future, and some minor refactoring.

Thanks @witgo for reporting this issue and working on the initial solution!

Author: witgo <witgo@qq.com>
Author: Aaron Davidson <aaron@databricks.com>

Closes #222 from aarondav/180 and squashes the following commits:

f74aad1 [Aaron Davidson] Avoid Option while generating call site & add unit tests
d2b4980 [witgo] Modify the position of the filter
1bc22d7 [witgo] Fix Stage.name return "apply at Option.scala:120"
2014-03-25 13:28:13 -07:00